You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2020/11/30 18:35:08 UTC

[asterixdb] branch master updated: [NO ISSUE] Add full-text stopwords filter

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e6986dd  [NO ISSUE] Add full-text stopwords filter
e6986dd is described below

commit e6986dd6b3ffeaff187160f2e1dbc665e31dcbe0
Author: Rui Guo <ru...@uci.edu>
AuthorDate: Wed Nov 25 19:50:06 2020 -0800

    [NO ISSUE] Add full-text stopwords filter
    
    Support stopwords filter when processing full-text search query.
    
    Users can configure full-text configurations and filters via SQLPP, and
    create full-text search index with a specific full-text configuration.
    Later, users can run ftcontains() search query and specify a full-text
    configuration.
    
    The full-text index will be utilized only if the index
    full-text configuration is the same as the query full-text configuration.
    If none of the index meets the requirement then a full-scan will run for
    the ftcontains() query.
    
    Change-Id: I6773a0d6c63693981aa1e29e9134e29391affe9a
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/8183
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 asterixdb/asterix-algebra/pom.xml                  |   5 +
 .../operators/physical/InvertedIndexPOperator.java |  21 +-
 .../jobgen/QueryLogicalExpressionJobGen.java       |   2 +
 .../asterix/optimizer/base/RuleCollections.java    |   4 +-
 ... FullTextContainsParameterCheckAndSetRule.java} |  93 ++++++--
 .../am/AbstractIntroduceAccessMethodRule.java      |  57 ++++-
 .../rules/am/InvertedIndexAccessMethod.java        |  28 +--
 .../asterix/optimizer/rules/util/FullTextUtil.java | 106 +++++++++
 asterixdb/asterix-app/pom.xml                      |   7 -
 .../asterix/app/io/PersistedResourceRegistry.java  |   6 +
 .../asterix/app/translator/QueryTranslator.java    | 255 ++++++++++++++++++++-
 .../apache/asterix/test/active/TestUserActor.java  |   2 +-
 .../asterix/test/runtime/NullMissingTest.java      |   8 +-
 .../basic/metadata_dataset/metadata_dataset.1.adm  |   2 +
 .../metadata_datatype/metadata_datatype.1.adm      |   3 +
 .../basic/metadata_index/metadata_index.1.adm      |   2 +
 .../metadata_selfjoin/metadata_selfjoin.1.adm      |   2 +
 .../stopwords-full-text-filter-1.1.ddl.sqlpp       |  28 +--
 .../stopwords-full-text-filter-1.10.query.sqlpp    |  23 +-
 .../stopwords-full-text-filter-1.11.query.sqlpp    |  24 +-
 .../stopwords-full-text-filter-1.12.query.sqlpp    |  24 +-
 .../stopwords-full-text-filter-1.13.query.sqlpp    |  24 +-
 .../stopwords-full-text-filter-1.14.ddl.sqlpp      |  25 +-
 .../stopwords-full-text-filter-1.15.query.sqlpp    |  22 +-
 .../stopwords-full-text-filter-1.2.update.sqlpp    |  34 ++-
 .../stopwords-full-text-filter-1.3.ddl.sqlpp       |  44 ++++
 .../stopwords-full-text-filter-1.4.query.sqlpp     |  20 +-
 .../stopwords-full-text-filter-1.5.query.sqlpp     |  20 +-
 .../stopwords-full-text-filter-1.6.ddl.sqlpp       |  25 +-
 .../stopwords-full-text-filter-1.7.query.sqlpp     |  20 +-
 .../stopwords-full-text-filter-1.8.update.sqlpp    |  37 ++-
 .../stopwords-full-text-filter-1.9.query.sqlpp     |  23 +-
 .../stopwords-full-text-filter-1.10.adm            |   1 +
 .../stopwords-full-text-filter-1.11.adm            |   2 +
 .../stopwords-full-text-filter-1.12.adm            |   1 +
 .../stopwords-full-text-filter-1.13.adm            |   2 +
 .../stopwords-full-text-filter-1.15.adm            |   1 +
 .../stopwords-full-text-filter-1.4.adm             |   2 +
 .../stopwords-full-text-filter-1.5.adm             |   2 +
 .../stopwords-full-text-filter-1.7.adm             |   4 +
 .../stopwords-full-text-filter-1.9.adm             |   1 +
 .../test/resources/runtimets/testsuite_sqlpp.xml   |   5 +
 .../asterix/common/api/IMetadataLockManager.java   |  60 +++++
 .../asterix/common/exceptions/ErrorCode.java       |   6 +
 .../asterix/common/metadata/IMetadataLockUtil.java |  24 +-
 .../src/main/resources/asx_errormsg/en.properties  |   5 +
 asterixdb/asterix-fuzzyjoin/pom.xml                |   4 +
 .../apache/asterix/lang/common/base/Statement.java |   4 +
 .../statement/CreateFullTextConfigStatement.java   | 102 +++++++++
 .../statement/CreateFullTextFilterStatement.java   | 105 +++++++++
 .../common/statement/CreateIndexStatement.java     |  11 +
 .../statement/FullTextConfigDropStatement.java     |  64 ++++++
 .../statement/FullTextFilterDropStatement.java     |  64 ++++++
 .../asterix/lang/common/util/FullTextUtil.java     |  77 +++++++
 .../lang/common/visitor/FormatPrintVisitor.java    |  32 +++
 .../base/AbstractQueryExpressionVisitor.java       |  24 ++
 .../lang/common/visitor/base/ILangVisitor.java     |  12 +
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj    | 133 ++++++++++-
 asterixdb/asterix-metadata/pom.xml                 |   6 +-
 .../org/apache/asterix/metadata/MetadataCache.java | 171 +++++++++++---
 .../apache/asterix/metadata/MetadataManager.java   | 142 +++++++++++-
 .../org/apache/asterix/metadata/MetadataNode.java  | 235 ++++++++++++++++++-
 .../metadata/MetadataTransactionContext.java       |  49 ++++
 .../asterix/metadata/api/IMetadataManager.java     |  77 +++++++
 .../apache/asterix/metadata/api/IMetadataNode.java |  76 ++++++
 .../metadata/bootstrap/MetadataBootstrap.java      |  42 +++-
 .../metadata/bootstrap/MetadataPrimaryIndexes.java |  15 ++
 .../metadata/bootstrap/MetadataRecordTypes.java    |  42 ++++
 .../metadata/declared/MetadataManagerUtil.java     |  19 ++
 .../metadata/declared/MetadataProvider.java        |  22 +-
 .../entities/FullTextConfigMetadataEntity.java     |  51 +++++
 .../entities/FullTextFilterMetadataEntity.java     |  46 ++++
 .../apache/asterix/metadata/entities/Index.java    |  17 +-
 ...ullTextConfigMetadataEntityTupleTranslator.java | 163 +++++++++++++
 ...ullTextFilterMetadataEntityTupleTranslator.java | 201 ++++++++++++++++
 .../IndexTupleTranslator.java                      |  27 ++-
 .../MetadataTupleTranslatorProvider.java           |   8 +
 .../asterix/metadata/lock/MetadataLockKey.java     |  10 +
 .../asterix/metadata/lock/MetadataLockManager.java |  32 +++
 .../asterix/metadata/utils/FullTextUtil.java       |  46 ++++
 .../InvertedIndexResourceFactoryProvider.java      |  10 +-
 .../asterix/metadata/utils/MetadataConstants.java  |   2 +
 .../asterix/metadata/utils/MetadataLockUtil.java   |  49 +++-
 ...aryCorrelatedInvertedIndexOperationsHelper.java |   8 +-
 .../SecondaryInvertedIndexOperationsHelper.java    |  10 +-
 .../IndexTupleTranslatorTest.java                  |   2 +-
 .../data/common/AOrderedListBinaryTokenizer.java   |  12 +
 asterixdb/asterix-runtime/pom.xml                  |   4 -
 ...java => FullTextContainsFunctionEvaluator.java} | 115 ++++++----
 ...ava => FullTextContainsFunctionDescriptor.java} |  62 ++++-
 ...xtContainsWithoutOptionFunctionDescriptor.java} |  54 ++---
 .../fulltext/AbstractFullTextFilterDescriptor.java |  29 +--
 .../runtime/fulltext/FullTextConfigDescriptor.java |  94 ++++++++
 .../fulltext/IFullTextConfigDescriptor.java        |  22 +-
 .../fulltext/IFullTextFilterDescriptor.java        |  40 ++++
 .../StopwordsFullTextFilterDescriptor.java         |  55 +++++
 .../runtime/functions/FunctionCollection.java      |   8 +-
 .../runtime/functions/FunctionTypeInferers.java    |  10 +
 .../hyracks-storage-am-lsm-invertedindex/pom.xml   |   9 +
 .../BinaryTokenizerOperatorDescriptor.java         |  21 +-
 .../BinaryTokenizerOperatorNodePushable.java       |  34 ++-
 .../dataflow/LSMInvertedIndexLocalResource.java    |  71 ++++--
 .../LSMInvertedIndexLocalResourceFactory.java      |  15 +-
 .../LSMInvertedIndexSearchOperatorDescriptor.java  |  18 +-
 ...LSMInvertedIndexSearchOperatorNodePushable.java |  16 +-
 .../AbstractFullTextFilterEvaluator.java}          |  29 +--
 .../AbstractFullTextFilterEvaluatorFactory.java}   |  27 +--
 .../fulltext/FullTextConfigEvaluator.java          | 145 ++++++++++++
 .../fulltext/FullTextConfigEvaluatorFactory.java   |  93 ++++++++
 .../FullTextFilterType.java}                       |  32 ++-
 .../IFullTextConfigEvaluator.java}                 |  32 ++-
 .../IFullTextConfigEvaluatorFactory.java}          |  20 +-
 .../IFullTextFilterEvaluator.java}                 |  24 +-
 .../IFullTextFilterEvaluatorFactory.java}          |  27 +--
 .../fulltext/StopwordsFullTextFilterEvaluator.java |  65 ++++++
 .../StopwordsFullTextFilterEvaluatorFactory.java   |  77 +++++++
 .../TokenizerCategory.java}                        |  24 +-
 .../lsm/invertedindex/impls/LSMInvertedIndex.java  |  14 +-
 .../impls/PartitionedLSMInvertedIndex.java         |  20 +-
 .../inmemory/InMemoryInvertedIndex.java            |   9 +-
 .../inmemory/InMemoryInvertedIndexOpContext.java   |  16 +-
 .../inmemory/PartitionedInMemoryInvertedIndex.java |  10 +-
 .../PartitionedInMemoryInvertedIndexOpContext.java |  10 +-
 .../search/AbstractTOccurrenceSearcher.java        |  17 +-
 .../search/InvertedIndexSearchPredicate.java       |  15 +-
 .../DelimitedUTF8StringBinaryTokenizer.java        |   8 +
 .../invertedindex/tokenizers/IBinaryTokenizer.java |  18 +-
 .../tokenizers/NGramUTF8StringBinaryTokenizer.java |   6 +
 .../util/InvertedIndexTokenizingTupleIterator.java |  26 ++-
 .../lsm/invertedindex/util/InvertedIndexUtils.java |  34 +--
 ...tionedInvertedIndexTokenizingTupleIterator.java |  19 +-
 .../pom.xml                                        |   4 +
 .../multithread/LSMInvertedIndexTestWorker.java    |   2 +
 .../util/LSMInvertedIndexTestContext.java          |  29 ++-
 .../util/LSMInvertedIndexTestUtils.java            |  33 ++-
 .../apache/hyracks/util/string/UTF8StringUtil.java |   4 +
 136 files changed, 4140 insertions(+), 760 deletions(-)

diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index cf5802f..9b9f253 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -231,5 +231,10 @@
       <groupId>it.unimi.dsi</groupId>
       <artifactId>fastutil</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-fuzzyjoin</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 491911b..fd664df 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -24,6 +24,7 @@ import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.FullTextUtil;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -59,6 +60,7 @@ import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
@@ -168,15 +170,20 @@ public class InvertedIndexPOperator extends IndexSearchPOperator {
                 InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
         IBinaryTokenizerFactory queryTokenizerFactory =
                 InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
+        IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+                FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider, secondaryIndex.getDataverseName(),
+                        secondaryIndex.getFullTextConfigName());
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
-        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
-                jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory,
-                retainInput, retainMissing, context.getMissingWriterFactory(),
-                dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
-                        IndexOperation.SEARCH, null),
-                minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
-                propagateIndexFilter, frameLimit);
+
+        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
+                new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField, dataflowHelperFactory,
+                        queryTokenizerFactory, fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput,
+                        retainMissing, context.getMissingWriterFactory(),
+                        dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+                                IndexOperation.SEARCH, null),
+                        minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+                        propagateIndexFilter, frameLimit);
         return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 74656e4..8707be5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -140,11 +140,13 @@ public class QueryLogicalExpressionJobGen implements ILogicalExpressionJobGen {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         IFunctionDescriptor fd = null;
         if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+            // Expr is an external function
             fd = ExternalFunctionDescriptorProvider
                     .getExternalFunctionDescriptor((IExternalFunctionInfo) expr.getFunctionInfo());
             CompilerProperties props = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
             FunctionTypeInferers.SET_ARGUMENTS_TYPE.infer(expr, fd, env, props);
         } else {
+            // Expr is an internal (built-in) function
             fd = resolveFunction(expr, env, context);
         }
         return fd.createEvaluatorFactory(args);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 4a2b629..8b261ee 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -46,7 +46,7 @@ import org.apache.asterix.optimizer.rules.ExtractWindowExpressionsRule;
 import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
 import org.apache.asterix.optimizer.rules.FindDataSourcesRule;
 import org.apache.asterix.optimizer.rules.FixReplicateOperatorOutputsRule;
-import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckRule;
+import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckAndSetRule;
 import org.apache.asterix.optimizer.rules.FuzzyEqRule;
 import org.apache.asterix.optimizer.rules.InjectTypeCastForFunctionArgumentsRule;
 import org.apache.asterix.optimizer.rules.InjectTypeCastForUnionRule;
@@ -176,7 +176,7 @@ public final class RuleCollections {
     }
 
     public static final List<IAlgebraicRewriteRule> buildFulltextContainsRuleCollection() {
-        return Collections.singletonList(new FullTextContainsParameterCheckRule());
+        return Collections.singletonList(new FullTextContainsParameterCheckAndSetRule());
     }
 
     public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection(ICcApplicationContext appCtx) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java
similarity index 77%
rename from asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
rename to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java
index c565b34..3e4e563 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java
@@ -23,12 +23,15 @@ import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.utils.FullTextUtil;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -45,11 +48,13 @@ import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionRef
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
+import com.google.common.base.Strings;
+
 /**
- * Checks whether the given parameters of the ftcontains() function are correct during the compilation.
+ * Checks whether the given parameters of the ftcontains() function are correct during the compilation,
+ * and fetch the full-text config from metadata which is necessary for the ftcontains() function
  */
-public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule {
-
+public class FullTextContainsParameterCheckAndSetRule implements IAlgebraicRewriteRule {
     // Visitor for checking and transforming ftcontains() expression
     protected FullTextContainsExpressionVisitor ftcontainsExprVisitor = new FullTextContainsExpressionVisitor();
 
@@ -59,7 +64,7 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
         if (context.checkIfInDontApplySet(this, opRef.getValue())) {
             return false;
         }
-        if (checkParameter(opRef, context)) {
+        if (checkAndSetParameter(opRef, context)) {
             OperatorPropertiesUtil.typeOpRec(opRef, context);
             return true;
         }
@@ -69,10 +74,14 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
     /**
      * Check the correctness of the parameters of the ftcontains(). Also rearrange options as arguments.
      * The expected form of ftcontains() is ftcontains(expression1, expression2, parameters as a record).
+     *
+     * If ftcontains() has the full-text config argument, this method will also fetch it (FullTextConfigDescriptor) from metadata
+     * and set it in the function expression so that the full-text config can be utilized later at run-time.
      */
-    private boolean checkParameter(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    private boolean checkAndSetParameter(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        ftcontainsExprVisitor.setContext(context);
         boolean modified = op.acceptExpressionTransform(ftcontainsExprVisitor);
         if (modified) {
             context.addToDontApplySet(this, op);
@@ -92,8 +101,13 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
         // The number of anticipated arguments for a full-text query when a user provide option(s) as a record.
         private static final int FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS = 3;
 
+        private IOptimizationContext context;
+
         public FullTextContainsExpressionVisitor() {
-            // no parameter is needed.
+        }
+
+        public void setContext(IOptimizationContext context) {
+            this.context = context;
         }
 
         @Override
@@ -156,14 +170,22 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                 // Sanity check for the types of the first two parameters
                 checkFirstAndSecondParamter(oldExprs, functionName);
 
+                // We cannot make ftConfigName a class-level variable in the visitor class
+                // because the visitor may be shared between multiple threads and such a variable may be corrupted
+                String ftConfigName = null;
                 // Checks and transforms the actual full-text parameters.
                 if (numberOfCorrectArguments == FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS) {
-                    checkValueForThirdParameter(oldExprs.get(2), newExprs, functionName);
+                    ftConfigName =
+                            checkValueForThirdParameterAndGetFullTextConfig(oldExprs.get(2), newExprs, functionName);
                 } else {
                     // no option provided case: sets the default option here.
                     setDefaultValueForThirdParameter(newExprs);
                 }
 
+                MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+                DataverseName dataverseName = metadataProvider.getDefaultDataverseName();
+                funcExpr.setOpaqueParameters(new Object[] { FullTextUtil
+                        .fetchFilterAndCreateConfigEvaluator(metadataProvider, dataverseName, ftConfigName) });
                 // Resets the last argument.
                 funcExpr.getArguments().clear();
                 funcExpr.getArguments().addAll(newExprs);
@@ -209,9 +231,14 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
          *
          * @param expr
          * @throws AlgebricksException
+         *
+         * @return the full-text config name if specified in the function option,
+         * null if not specified which implies the default full-text config will be utilized later
          */
-        private void checkValueForThirdParameter(Mutable<ILogicalExpression> expr,
+        private String checkValueForThirdParameterAndGetFullTextConfig(Mutable<ILogicalExpression> expr,
                 List<Mutable<ILogicalExpression>> newArgs, String functionName) throws AlgebricksException {
+            String ftConfigName = null;
+
             // Get the last parameter - this should be a record-constructor.
             AbstractFunctionCallExpression openRecConsExpr = (AbstractFunctionCallExpression) expr.getValue();
             FunctionIdentifier openRecConsFi = openRecConsExpr.getFunctionIdentifier();
@@ -222,11 +249,17 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
             }
 
             // We multiply 2 because the layout of the arguments are: [expr, val, expr1, val1, ...]
-            if (openRecConsExpr.getArguments().size() > FullTextContainsDescriptor.getParamTypeMap().size() * 2) {
+            if (openRecConsExpr.getArguments().size() > FullTextContainsFunctionDescriptor.getParamTypeMap().size()
+                    * 2) {
                 throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION,
                         openRecConsExpr.getSourceLocation(), functionName);
             }
 
+            if (openRecConsExpr.getArguments().size() % 2 != 0) {
+                throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
+                        openRecConsExpr.getSourceLocation(), functionName);
+            }
+
             for (int i = 0; i < openRecConsExpr.getArguments().size(); i = i + 2) {
                 ILogicalExpression optionExpr = openRecConsExpr.getArguments().get(i).getValue();
                 ILogicalExpression optionExprVal = openRecConsExpr.getArguments().get(i + 1).getValue();
@@ -239,16 +272,15 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                 }
 
                 option = option.toLowerCase();
-                if (!FullTextContainsDescriptor.getParamTypeMap().containsKey(option)) {
+                if (!FullTextContainsFunctionDescriptor.getParamTypeMap().containsKey(option)) {
                     throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExprVal.getSourceLocation(),
                             functionName, option);
                 }
 
                 String optionTypeStringVal = null;
-
                 // If the option value is a constant, then we can check here.
                 if (optionExprVal.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                    switch (FullTextContainsDescriptor.getParamTypeMap().get(option)) {
+                    switch (FullTextContainsFunctionDescriptor.getParamTypeMap().get(option)) {
                         case STRING:
                             optionTypeStringVal = ConstantExpressionUtil.getStringConstant(optionExprVal);
                             if (optionTypeStringVal == null) {
@@ -265,9 +297,14 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
 
                     // Check the validity of option value
                     switch (option) {
-                        case FullTextContainsDescriptor.SEARCH_MODE_OPTION:
+                        case FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION:
                             checkSearchModeOption(optionTypeStringVal, functionName, optionExprVal.getSourceLocation());
                             break;
+                        case FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION:
+                            checkFullTextConfigOption(optionTypeStringVal, functionName,
+                                    optionExprVal.getSourceLocation());
+                            ftConfigName = optionTypeStringVal;
+                            break;
                         default:
                             throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
                                     optionExprVal.getSourceLocation(), functionName, option);
@@ -278,31 +315,45 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                 newArgs.add(new MutableObject<ILogicalExpression>(optionExpr));
                 newArgs.add(new MutableObject<ILogicalExpression>(optionExprVal));
             }
+            return ftConfigName;
         }
 
         private void checkSearchModeOption(String optionVal, String functionName, SourceLocation sourceLoc)
                 throws AlgebricksException {
-            if (optionVal.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)
-                    || optionVal.equals(FullTextContainsDescriptor.DISJUNCTIVE_SEARCH_MODE_OPTION)) {
+            if (optionVal.equals(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())
+                    || optionVal.equals(FullTextContainsFunctionDescriptor.SearchMode.ANY.getValue())) {
                 return;
             } else {
                 throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, functionName, optionVal);
             }
         }
 
+        private void checkFullTextConfigOption(String optionVal, String functionName, SourceLocation sourceLoc)
+                throws AlgebricksException {
+            // Currently, here we only check if the full-text config is null or empty string
+            // We will check if the full-text config exists at run time
+            if (Strings.isNullOrEmpty(optionVal)) {
+                throw CompilationException.create(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, functionName,
+                        FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION, "not-null", "null");
+            } else {
+                return;
+            }
+        }
+
         /**
          * Sets the default option value(s) when a user doesn't provide any option.
          */
-        void setDefaultValueForThirdParameter(List<Mutable<ILogicalExpression>> newArgs) throws AlgebricksException {
+        void setDefaultValueForThirdParameter(List<Mutable<ILogicalExpression>> newArgs) {
             // Sets the search mode option: the default option is conjunctive search.
             ILogicalExpression searchModeOptionExpr = new ConstantExpression(
-                    new AsterixConstantValue(new AString(FullTextContainsDescriptor.SEARCH_MODE_OPTION)));
-            ILogicalExpression searchModeValExpr = new ConstantExpression(
-                    new AsterixConstantValue(new AString(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)));
-
+                    new AsterixConstantValue(new AString(FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION)));
+            ILogicalExpression searchModeValExpr = new ConstantExpression(new AsterixConstantValue(
+                    new AString(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())));
             // Add this option as arguments to the ftcontains().
             newArgs.add(new MutableObject<ILogicalExpression>(searchModeOptionExpr));
             newArgs.add(new MutableObject<ILogicalExpression>(searchModeValExpr));
+
+            // We don't set the full-text config option here because the default value should be null
         }
 
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 6b045ea..9c39f3a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -53,6 +53,7 @@ import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
+import org.apache.asterix.optimizer.rules.util.FullTextUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -80,6 +81,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.Var
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
 
 /**
@@ -214,7 +216,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     /**
      * Choose all indexes that match the given access method. These indexes will be used as index-search
      * to replace the given predicates in a SELECT operator. Also, if there are multiple same type of indexes
-     * on the same field, only of them will be chosen. Allowed cases (AccessMethod, IndexType) are:
+     * on the same field, only one of them will be chosen. Allowed cases (AccessMethod, IndexType) are:
      * [BTreeAccessMethod , IndexType.BTREE], [RTreeAccessMethod , IndexType.RTREE],
      * [InvertedIndexAccessMethod, IndexType.SINGLE_PARTITION_WORD_INVIX || SINGLE_PARTITION_NGRAM_INVIX ||
      * LENGTH_PARTITIONED_WORD_INVIX || LENGTH_PARTITIONED_NGRAM_INVIX]
@@ -235,13 +237,34 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 IAccessMethod chosenAccessMethod = amEntry.getKey();
                 Index chosenIndex = indexEntry.getKey();
                 IndexType indexType = chosenIndex.getIndexType();
-                boolean isKeywordOrNgramIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
-                        || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
-                        || indexType == IndexType.SINGLE_PARTITION_WORD_INVIX
+                boolean isKeywordIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+                        || indexType == IndexType.SINGLE_PARTITION_WORD_INVIX;
+                boolean isNgramIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
                         || indexType == IndexType.SINGLE_PARTITION_NGRAM_INVIX;
                 if ((chosenAccessMethod == BTreeAccessMethod.INSTANCE && indexType == IndexType.BTREE)
                         || (chosenAccessMethod == RTreeAccessMethod.INSTANCE && indexType == IndexType.RTREE)
-                        || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isKeywordOrNgramIndexChosen)) {
+                        // the inverted index will be utilized
+                        // For Ngram, the full-text config used in the index and in the query are always the default one,
+                        // so we don't check if the full-text config in the index and query match
+                        //
+                        // Note that the ngram index can be used in both
+                        // 1) full-text ftcontains() function
+                        // 2) non-full-text, regular string contains() function
+                        // 3) edit-distance functions that take keyword as an argument,
+                        //     e.g. edit_distance_check() when the threshold is larger than 1
+                        || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isNgramIndexChosen)
+                        // the inverted index will be utilized
+                        // For keyword, different full-text configs may apply to different indexes on the same field,
+                        // so we need to check if the config used in the index matches the config in the ftcontains() query
+                        // If not, then we cannot use this index.
+                        //
+                        // Note that for now, the keyword/fulltext index can be utilized in
+                        // 1) the full-text ftcontains() function
+                        // 2) functions that take keyword as an argument, e.g. edit_distance_check() when the threshold is 1
+                        || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isKeywordIndexChosen
+                                && isSameFullTextConfigInIndexAndQuery(analysisCtx,
+                                        chosenIndex.getFullTextConfigName()))) {
+
                     if (resultVarsToIndexTypesMap.containsKey(indexEntry.getValue())) {
                         List<IndexType> appliedIndexTypes = resultVarsToIndexTypesMap.get(indexEntry.getValue());
                         if (!appliedIndexTypes.contains(indexType)) {
@@ -260,6 +283,30 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         return result;
     }
 
+    private boolean isSameFullTextConfigInIndexAndQuery(AccessMethodAnalysisContext analysisCtx,
+            String indexFullTextConfig) {
+        IOptimizableFuncExpr expr = analysisCtx.getMatchedFuncExpr(0);
+        if (FullTextUtil.isFullTextContainsFunctionExpr(expr)) {
+            // ftcontains()
+            String expectedConfig = FullTextUtil.getFullTextConfigNameFromExpr(expr);
+            if (Strings.isNullOrEmpty(expectedConfig)) {
+                return Strings.isNullOrEmpty(indexFullTextConfig);
+            } else if (expectedConfig.equals(indexFullTextConfig)) {
+                return true;
+            }
+        } else {
+            // besides ftcontains(), there are other functions that utilize the full-text inverted-index,
+            // e.g. edit_distance_check(),
+            // for now, we don't accept users to specify the full-text config in those functions,
+            // that means, we assume the full-text config used in those function is always the default one with the name null,
+            // and if the index full-text config name is also null, the index can be utilized
+            if (Strings.isNullOrEmpty(indexFullTextConfig)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
     /**
      * Removes irrelevant access methods candidates, based on whether the
      * expressions in the query match those in the index. For example, some
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 1c151a1..98ad10a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -50,8 +50,7 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
+import org.apache.asterix.optimizer.rules.util.FullTextUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -886,7 +885,8 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         jobGenParams.setSearchKeyType(typeTag);
     }
 
-    private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
+    private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams)
+            throws CompilationException {
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS) {
             jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
             jobGenParams.setSimilarityThreshold(new AsterixConstantValue(AMissing.MISSING));
@@ -921,32 +921,12 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             // We check the last argument of the given full-text search to see whether conjunctive or disjunctive
             // search parameter is given. This is the last argument of the function call expression.
             AbstractFunctionCallExpression funcExpr = optFuncExpr.getFuncExpr();
-            jobGenParams.setSearchModifierType(getFullTextOption(funcExpr));
+            jobGenParams.setSearchModifierType(FullTextUtil.getFullTextSearchModeFromExpr(funcExpr));
 
             jobGenParams.setSimilarityThreshold(new AsterixConstantValue(ANull.NULL));
         }
     }
 
-    private static SearchModifierType getFullTextOption(AbstractFunctionCallExpression funcExpr) {
-        if (funcExpr.getArguments().size() < 3 || funcExpr.getArguments().size() % 2 != 0) {
-            // If no parameters or incorrect number of parameters are given, the default search type is returned.
-            return SearchModifierType.CONJUNCTIVE;
-        }
-        // From the third argument, it contains full-text search options.
-        for (int i = 2; i < funcExpr.getArguments().size(); i = i + 2) {
-            String optionName = ConstantExpressionUtil.getStringArgument(funcExpr, i);
-            if (optionName.equals(FullTextContainsDescriptor.SEARCH_MODE_OPTION)) {
-                String searchType = ConstantExpressionUtil.getStringArgument(funcExpr, i + 1);
-                if (searchType.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)) {
-                    return SearchModifierType.CONJUNCTIVE;
-                } else {
-                    return SearchModifierType.DISJUNCTIVE;
-                }
-            }
-        }
-        return null;
-    }
-
     private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList,
             ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context)
             throws AlgebricksException {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java
new file mode 100644
index 0000000..ab62e85
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.optimizer.rules.am.IOptimizableFuncExpr;
+import org.apache.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class FullTextUtil {
+
+    public static boolean isFullTextContainsFunctionExpr(IOptimizableFuncExpr expr) {
+        return isFullTextContainsFunctionExpr(expr.getFuncExpr());
+    }
+
+    public static boolean isFullTextContainsFunctionExpr(AbstractFunctionCallExpression expr) {
+        FunctionIdentifier funcId = expr.getFunctionIdentifier();
+        if (funcId.equals(BuiltinFunctions.FULLTEXT_CONTAINS)
+                || funcId.equals(BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION)) {
+            return true;
+        }
+        return false;
+    }
+
+    // If not a full-text function expression, then return null
+    // Otherwise, return the full-text config if one exists in the expression, otherwise return the default config
+    public static String getFullTextConfigNameFromExpr(IOptimizableFuncExpr expr) {
+        return getFullTextConfigNameFromExpr(expr.getFuncExpr());
+    }
+
+    // ToDo: here we are parsing the expr manually, maybe we can find a better way to parse the arguments,
+    //  e.g. convert the argument into an AdmObjectNode and then read from the object node
+    public static String getFullTextConfigNameFromExpr(AbstractFunctionCallExpression funcExpr) {
+        if (isFullTextContainsFunctionExpr(funcExpr) == false) {
+            return null;
+        }
+
+        String configName = null;
+        List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
+
+        // The first two arguments are
+        // 1) the full-text record field to be queried,
+        // 2) the query keyword array
+        // The next fields are the list of full-text search options,
+        // say, the next 4 fields can be "mode", "all", "config", "my_full_text_config"
+        // Originally, the full-text search option is an Asterix record such as
+        //     {"mode": "all", "config": "my_full_text_config"}
+        for (int i = 2; i < arguments.size(); i += 2) {
+            // The the full-text search option arguments are already checked in FullTextContainsParameterCheckAndSetRule,
+            String optionName = ConstantExpressionUtil.getStringConstant(arguments.get(i).getValue());
+
+            if (optionName.equalsIgnoreCase(FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION)) {
+                configName = ConstantExpressionUtil.getStringConstant(arguments.get(i + 1).getValue());
+                break;
+            }
+        }
+
+        return configName;
+    }
+
+    public static InvertedIndexAccessMethod.SearchModifierType getFullTextSearchModeFromExpr(
+            AbstractFunctionCallExpression funcExpr) {
+
+        // After the third argument, the following arguments are full-text search options.
+        for (int i = 2; i < funcExpr.getArguments().size(); i = i + 2) {
+            String optionName = ConstantExpressionUtil.getStringArgument(funcExpr, i);
+
+            if (optionName.equals(FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION)) {
+                String searchType = ConstantExpressionUtil.getStringArgument(funcExpr, i + 1);
+
+                if (searchType.equals(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())) {
+                    return InvertedIndexAccessMethod.SearchModifierType.CONJUNCTIVE;
+                } else {
+                    return InvertedIndexAccessMethod.SearchModifierType.DISJUNCTIVE;
+                }
+            }
+        }
+
+        // Use CONJUNCTIVE by default
+        return InvertedIndexAccessMethod.SearchModifierType.CONJUNCTIVE;
+    }
+
+}
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 7f8085d..8b5fed1 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -154,7 +154,6 @@
           </ignoredUsedUndeclaredDependencies>
           <usedDependencies combine.children="append">
             <usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
-            <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
             <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
           </usedDependencies>
           <ignoredUnusedDeclaredDependencies>
@@ -756,12 +755,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-fuzzyjoin</artifactId>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-geo</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 1168567..4cc6e43 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -111,6 +111,8 @@ import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResource;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.StopwordsFullTextFilterEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
@@ -273,6 +275,10 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
         registeredClasses.put("DelimitedUTF8StringBinaryTokenizerFactory",
                 DelimitedUTF8StringBinaryTokenizerFactory.class);
 
+        // IFullTextConfigFactory
+        registeredClasses.put("FullTextConfigEvaluatorFactory", FullTextConfigEvaluatorFactory.class);
+        registeredClasses.put("StopwordsFullTextFilterEvaluatorFactory", StopwordsFullTextFilterEvaluatorFactory.class);
+
         // ITokenFactory
         registeredClasses.put("AListElementTokenFactory", AListElementTokenFactory.class);
         registeredClasses.put("HashedUTF8NGramTokenFactory", HashedUTF8NGramTokenFactory.class);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 607e23a..9137f5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.translator;
 
+import static org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement.FIELD_TYPE_STOPWORDS;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.InputStream;
@@ -106,6 +108,8 @@ import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -119,6 +123,8 @@ import org.apache.asterix.lang.common.statement.DropDatasetStatement;
 import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
 import org.apache.asterix.lang.common.statement.FeedDropStatement;
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -157,6 +163,8 @@ import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -179,6 +187,10 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
 import org.apache.asterix.translator.ClientRequest;
@@ -228,10 +240,14 @@ import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+
 /*
  * Provides functionality for executing a batch of Query statements (queries included)
  * sequentially.
@@ -338,6 +354,12 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     case CREATE_INDEX:
                         handleCreateIndexStatement(metadataProvider, stmt, hcc, requestParameters);
                         break;
+                    case CREATE_FULL_TEXT_FILTER:
+                        handleCreateFullTextFilterStatement(metadataProvider, stmt);
+                        break;
+                    case CREATE_FULL_TEXT_CONFIG:
+                        handleCreateFullTextConfigStatement(metadataProvider, stmt);
+                        break;
                     case TYPE_DECL:
                         handleCreateTypeStatement(metadataProvider, stmt);
                         break;
@@ -353,6 +375,12 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     case INDEX_DROP:
                         handleIndexDropStatement(metadataProvider, stmt, hcc, requestParameters);
                         break;
+                    case FULL_TEXT_FILTER_DROP:
+                        handleFullTextFilterDrop(metadataProvider, stmt, hcc, requestParameters);
+                        break;
+                    case FULL_TEXT_CONFIG_DROP:
+                        handleFullTextConfigDrop(metadataProvider, stmt, hcc, requestParameters);
+                        break;
                     case TYPE_DROP:
                         handleTypeDropStatement(metadataProvider, stmt);
                         break;
@@ -970,7 +998,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
         validateDatabaseObjectName(indexName, stmt.getSourceLocation());
-        lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+        String fullTextConfigName = stmtCreateIndex.getFullTextConfigName();
+        lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
+                fullTextConfigName);
         try {
             doCreateIndex(metadataProvider, stmtCreateIndex, dataverseName, datasetName, hcc, requestParameters);
         } finally {
@@ -1116,9 +1146,10 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             validateIndexKeyFields(stmtCreateIndex, keySourceIndicators, aRecordType, metaRecordType, indexFields,
                     indexFieldTypes);
 
-            Index newIndex = new Index(dataverseName, datasetName, indexName, indexType, indexFields,
-                    keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(), overridesFieldTypes,
-                    stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
+            Index newIndex =
+                    new Index(dataverseName, datasetName, indexName, indexType, indexFields, keySourceIndicators,
+                            indexFieldTypes, stmtCreateIndex.getGramLength(), stmtCreateIndex.getFullTextConfigName(),
+                            overridesFieldTypes, stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
 
             bActiveTxn = false; // doCreateIndexImpl() takes over the current transaction
             doCreateIndexImpl(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
@@ -1131,6 +1162,133 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
+    public void handleCreateFullTextFilterStatement(MetadataProvider metadataProvider, Statement stmt)
+            throws Exception {
+        CreateFullTextFilterStatement stmtCreateFilter = (CreateFullTextFilterStatement) stmt;
+        DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
+        String fullTextFilterName = stmtCreateFilter.getFilterName();
+        validateDatabaseObjectName(fullTextFilterName, stmt.getSourceLocation());
+
+        lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
+        try {
+            doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected void doCreateFullTextFilter(MetadataProvider metadataProvider,
+            CreateFullTextFilterStatement stmtCreateFilter, DataverseName dataverseName) throws Exception {
+        AbstractFullTextFilterDescriptor filterDescriptor;
+
+        String filterType = stmtCreateFilter.getFilterType();
+        if (filterType == null) {
+            throw new CompilationException(ErrorCode.PARSE_ERROR, stmtCreateFilter.getSourceLocation(),
+                    "full-text filter type is null");
+        }
+
+        switch (filterType) {
+            case FIELD_TYPE_STOPWORDS: {
+                filterDescriptor = new StopwordsFullTextFilterDescriptor(dataverseName,
+                        stmtCreateFilter.getFilterName(), stmtCreateFilter.getStopwordsList());
+                break;
+            }
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, stmtCreateFilter.getSourceLocation(),
+                        "Unexpected full-text filter type: " + filterType);
+        }
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, stmtCreateFilter.getSourceLocation(),
+                        dataverseName);
+            }
+
+            String filterName = stmtCreateFilter.getFilterName();
+            FullTextFilterMetadataEntity existingFilter =
+                    MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, filterName);
+            if (existingFilter != null) {
+                if (stmtCreateFilter.getIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_ALREADY_EXISTS,
+                            stmtCreateFilter.getSourceLocation(), filterName);
+                }
+            }
+
+            MetadataManager.INSTANCE.addFullTextFilter(mdTxnCtx, new FullTextFilterMetadataEntity(filterDescriptor));
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
+    public void handleCreateFullTextConfigStatement(MetadataProvider metadataProvider, Statement stmt)
+            throws Exception {
+        CreateFullTextConfigStatement stmtCreateConfig = (CreateFullTextConfigStatement) stmt;
+        DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
+        String configName = stmtCreateConfig.getConfigName();
+        validateDatabaseObjectName(configName, stmt.getSourceLocation());
+        ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
+
+        lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
+                filterNames);
+        try {
+            doCreateFullTextConfig(metadataProvider, stmtCreateConfig, dataverseName, configName, filterNames);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected void doCreateFullTextConfig(MetadataProvider metadataProvider,
+            CreateFullTextConfigStatement stmtCreateConfig, DataverseName dataverseName, String configName,
+            ImmutableList<String> filterNames) throws Exception {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+        try {
+            FullTextConfigMetadataEntity existingConfig =
+                    MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, configName);
+            if (existingConfig != null) {
+                if (stmtCreateConfig.getIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new CompilationException(ErrorCode.FULL_TEXT_CONFIG_ALREADY_EXISTS,
+                            stmtCreateConfig.getSourceLocation(), configName);
+                }
+            }
+
+            ImmutableList.Builder<IFullTextFilterDescriptor> filterDescriptorsBuilder =
+                    ImmutableList.<IFullTextFilterDescriptor> builder();
+            for (String filterName : filterNames) {
+                FullTextFilterMetadataEntity filterMetadataEntity =
+                        MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, filterName);
+                if (filterMetadataEntity == null) {
+                    throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_NOT_FOUND,
+                            stmtCreateConfig.getSourceLocation(), filterName);
+                }
+            }
+
+            TokenizerCategory tokenizerCategory = stmtCreateConfig.getTokenizerCategory();
+            FullTextConfigDescriptor configDescriptor =
+                    new FullTextConfigDescriptor(dataverseName, configName, tokenizerCategory, filterNames);
+            FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(configDescriptor);
+
+            MetadataManager.INSTANCE.addFullTextConfig(mdTxnCtx, configMetadataEntity);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
     private void doCreateIndexImpl(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Dataset ds,
             Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc) throws Exception {
         ProgressState progress = ProgressState.NO_PROGRESS;
@@ -1907,6 +2065,95 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
+    protected void handleFullTextFilterDrop(MetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+        FullTextFilterDropStatement stmtFilterDrop = (FullTextFilterDropStatement) stmt;
+        DataverseName dataverseName = getActiveDataverseName(stmtFilterDrop.getDataverseName());
+        String fullTextFilterName = stmtFilterDrop.getFilterName();
+
+        lockUtil.dropFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
+        try {
+            doDropFullTextFilter(metadataProvider, stmtFilterDrop, dataverseName, fullTextFilterName);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected void doDropFullTextFilter(MetadataProvider metadataProvider, FullTextFilterDropStatement stmtFilterDrop,
+            DataverseName dataverseName, String fullTextFilterName) throws AlgebricksException, RemoteException {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            FullTextFilterMetadataEntity filter =
+                    MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, fullTextFilterName);
+            if (filter == null) {
+                if (stmtFilterDrop.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_NOT_FOUND,
+                            stmtFilterDrop.getSourceLocation(), fullTextFilterName);
+                }
+            }
+
+            MetadataManager.INSTANCE.dropFullTextFilter(mdTxnCtx, dataverseName, fullTextFilterName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
+    protected void handleFullTextConfigDrop(MetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc, IRequestParameters requestParameters)
+            throws AlgebricksException, RemoteException {
+        FullTextConfigDropStatement stmtConfigDrop = (FullTextConfigDropStatement) stmt;
+        DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
+        String configName = stmtConfigDrop.getConfigName();
+
+        lockUtil.dropFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName);
+        try {
+            doDropFullTextConfig(metadataProvider, stmtConfigDrop, hcc, requestParameters);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    private void doDropFullTextConfig(MetadataProvider metadataProvider, FullTextConfigDropStatement stmtConfigDrop,
+            IHyracksClientConnection hcc, IRequestParameters requestParameters)
+            throws RemoteException, AlgebricksException {
+        // If the config name is null, then it means the default config
+        if (Strings.isNullOrEmpty(stmtConfigDrop.getConfigName())) {
+            throw new CompilationException(ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED,
+                    stmtConfigDrop.getSourceLocation());
+        }
+
+        DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        String fullTextConfigName = stmtConfigDrop.getConfigName();
+
+        try {
+            FullTextConfigMetadataEntity configMetadataEntity =
+                    MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, fullTextConfigName);
+            if (configMetadataEntity == null) {
+                if (stmtConfigDrop.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new CompilationException(ErrorCode.FULL_TEXT_CONFIG_NOT_FOUND,
+                            stmtConfigDrop.getSourceLocation(), fullTextConfigName);
+                }
+            }
+
+            MetadataManager.INSTANCE.dropFullTextConfig(mdTxnCtx, dataverseName, fullTextConfigName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
     protected void handleTypeDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
         SourceLocation sourceLoc = stmtTypeDrop.getSourceLocation();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
index bc2c110..2d64f79 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
@@ -200,7 +200,7 @@ public class TestUserActor extends Actor {
                 DataverseName dataverseName = dataset.getDataverseName();
                 String datasetName = dataset.getDatasetName();
                 try {
-                    lockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName);
+                    lockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName, null);
                     if (actionListener.isActive()) {
                         throw new RuntimeDataException(ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
                                 DatasetUtil.getFullyQualifiedDisplayName(dataverseName, datasetName) + ".index",
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
index 55a1a80..8cd02ed 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
@@ -86,9 +86,11 @@ public class NullMissingTest {
             // Include only functions annotated with MissingNullInOutFunction
             if (functionDescriptor.getClass().isAnnotationPresent(MissingNullInOutFunction.class)) {
 
-                // We test all functions except record and cast functions, which requires type settings (we test them
-                // in runtime tests).
-                if (!className.contains("record") && !className.contains("Cast")) {
+                // We test all functions except record, cast and full-text contains functions,
+                // which requires type settings or argument settings.
+                // Instead, we test them in runtime tests.
+                if (!className.contains("record") && !className.contains("Cast")
+                        && !className.contains("FullTextContains")) {
                     tests.add(new Object[] { getTestName(functionDescriptor.getClass()), functionDescriptor });
                 } else {
                     LOGGER.log(Level.INFO, "Excluding " + className);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 84ce315..80bc98a 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -7,6 +7,8 @@
 { "DataverseName": "Metadata", "DatasetName": "Feed", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": " [...]
 { "DataverseName": "Metadata", "DatasetName": "FeedConnection", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails":  [...]
 { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileS [...]
+{ "DataverseName": "Metadata", "DatasetName": "FullTextConfig", "DatatypeDataverseName": "Metadata", "DatatypeName": "FullTextConfigRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails":  [...]
+{ "DataverseName": "Metadata", "DatasetName": "FullTextFilter", "DatatypeDataverseName": "Metadata", "DatatypeName": "FullTextFilterRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails":  [...]
 { "DataverseName": "Metadata", "DatasetName": "Function", "DatatypeDataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStruc [...]
 { "DataverseName": "Metadata", "DatasetName": "Index", "DatatypeDataverseName": "Metadata", "DatatypeName": "IndexRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": [...]
 { "DataverseName": "Metadata", "DatasetName": "Library", "DatatypeDataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructu [...]
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 9aa9269..d352a6f 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -29,6 +29,9 @@
 { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false, "IsMissable": false }, { "Fiel [...]
 { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
 { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextConfigRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextConfigName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Tokenizer", "FieldType": "string", "IsNullable": true, "IsMissable": false }, { "FieldName": "FullTextFilt [...]
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextFilterPipeline", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Nov 23 02:42:02 PST 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextFilterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextFilterName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextFilterType", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": [...]
 { "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Params", "FieldType": "FunctionReco [...]
 { "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
 { "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index bdb22c7..5a40088 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -7,6 +7,8 @@
 { "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "FeedConnection", "IndexName": "FeedConnection", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextConfig", "IndexName": "FullTextConfig", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FullTextConfigName" ] ], "IsPrimary": true, "Timestamp": "Thu Oct 29 16:20:12 PDT 2020", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextFilter", "IndexName": "FullTextFilter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FullTextFilterName" ] ], "IsPrimary": true, "Timestamp": "Thu Oct 29 16:20:12 PDT 2020", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
 { "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
index 7031c72..27f8a71 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
@@ -12,4 +12,6 @@
 { "dv1": "Metadata", "dv2": "Metadata" }
 { "dv1": "Metadata", "dv2": "Metadata" }
 { "dv1": "Metadata", "dv2": "Metadata" }
+{ "dv1": "Metadata", "dv2": "Metadata" }
+{ "dv1": "Metadata", "dv2": "Metadata" }
 { "dv1": "Metadata", "dv2": "Metadata" }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp
index 6a7da02..ad34b86 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp
@@ -17,22 +17,18 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 1: Create dataset and insert data
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+DROP DATAVERSE MyDataVerse IF EXISTS;
+CREATE DATAVERSE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+USE MyDataVerse;
 
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+DROP DATASET MyMessageDataset IF EXISTS;
+CREATE TYPE MyMessageType AS {
+    myMessageId:   int,
+    myMessageBody: string
+};
+CREATE DATASET MyMessageDataset(MyMessageType)
+    PRIMARY KEY myMessageId;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp
index 6a7da02..1c6752d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp
@@ -17,22 +17,11 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 5: Select with ft_config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+// Without any ftcontains option
+SELECT VALUE myMessage from MyMessageDataset myMessage
+    WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"]) ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp
index 6a7da02..17f3024 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp
@@ -17,22 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 5a: Select with ft_config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+// With FT Config my_first_stopword_config and `the`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+    WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"], {"mode":"all", "config": "my_first_stopword_config"})
+    ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp
index 6a7da02..6a05ae6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp
@@ -17,22 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 5b: Select with ft_config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+// With FT Config my_first_stopword_config and `xxx`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+    WHERE ftcontains(myMessage.myMessageBody, ["xxx", "cat"], {"mode":"all", "config": "my_first_stopword_config"})
+    ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp
index 6a7da02..c05bd04 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp
@@ -17,22 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 5c: Select with ft_config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+// With FT Config my_second_stopword_config and `yyy`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+    WHERE ftcontains(myMessage.myMessageBody, ["yyy", "cat"], {"mode":"all", "config": "my_second_stopword_config"})
+    ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp
index 6a7da02..2c10e35 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp
@@ -17,22 +17,13 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+/////////////////////////////////////////
+// Step 6: Do a full-scan when no index with the same config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+DROP FULLTEXT CONFIG my_third_stopword_config IF EXISTS;
+CREATE FULLTEXT CONFIG my_third_stopword_config IF NOT EXISTS AS {
+    "tokenizer": "Word",
+    "filterPipeline": []
+}; // No index created with this new config
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp
index 6a7da02..8d5ff33 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp
@@ -17,22 +17,10 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+/////////////////////////////////////////
+// Step 6: Do a full-scan when no index with the same config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+SELECT count(myMessage) from MyMessageDataset myMessage
+   WHERE ftcontains(myMessage.myMessageBody, ["xxx", "smart", "black"], {"mode":"all", "config": "my_third_stopword_config"});
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp
index 6a7da02..13b4250 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp
@@ -17,22 +17,18 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
-
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
-
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+USE MyDataVerse;
+
+INSERT INTO MyMessageDataset ([
+  {
+      "myMessageId":    1,
+      "myMessageBody":  "a smart black dog"
+  }
+]);
+
+UPSERT INTO MyMessageDataset ([
+  {
+      "myMessageId":    2,
+      "myMessageBody":  "the smart black dog"
+  }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp
new file mode 100644
index 0000000..e475c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp
@@ -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.
+ */
+
+USE MyDataVerse;
+
+DROP FULLTEXT CONFIG my_stopword_config IF EXISTS;
+DROP FULLTEXT CONFIG my_first_stopword_config IF EXISTS;
+DROP FULLTEXT CONFIG my_second_stopword_config IF EXISTS;
+DROP FULLTEXT FILTER my_first_stopword_filter IF EXISTS;
+DROP FULLTEXT FILTER my_second_stopword_filter IF EXISTS;
+
+CREATE FULLTEXT FILTER my_first_stopword_filter IF NOT EXISTS AS {
+    "type": "stopwords",
+    "stopwordsList": ["a", "an", "the"]
+};
+CREATE FULLTEXT CONFIG my_first_stopword_config IF NOT EXISTS AS {
+    "tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+    "filterPipeline": ["my_first_stopword_filter"]
+};
+
+CREATE FULLTEXT FILTER my_second_stopword_filter IF NOT EXISTS AS {
+    "type": "stopwords",
+    "stopwordsList": ["xxx", "yyy", "zzz"]
+};
+CREATE FULLTEXT CONFIG my_second_stopword_config IF NOT EXISTS AS {
+    "tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+    "filterPipeline": ["my_second_stopword_filter"]
+};
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp
index 6a7da02..b5c1363 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp
@@ -17,22 +17,4 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
-
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
-
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+SELECT VALUE ix FROM Metadata.`FullTextConfig` ix ORDER BY ix.FullTextConfigName;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp
index 6a7da02..8e0d694 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp
@@ -17,22 +17,4 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
-
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
-
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+SELECT VALUE ix FROM Metadata.`FullTextFilter` ix ORDER BY ix.FullTextFilterName;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp
similarity index 56%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp
index 6a7da02..fe59571 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp
@@ -17,22 +17,15 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 3: Create 3 FullText Indices
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+DROP INDEX MyMessageDataset.message_ft_index_0 IF EXISTS;
+DROP INDEX MyMessageDataset.message_ft_index_1 IF EXISTS;
+DROP INDEX MyMessageDataset.message_ft_index_2 IF EXISTS;
 
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+CREATE INDEX message_ft_index_0  IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT;
+CREATE INDEX message_ft_index_1  IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT USING my_first_stopword_config;
+CREATE INDEX message_ft_index_2  IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT USING my_second_stopword_config;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp
index 6a7da02..f770fee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp
@@ -17,22 +17,4 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
-
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
-
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+SELECT Value v from Metadata.`Index` v WHERE v.DataverseName = "MyDataVerse" ORDER BY v. IndexName;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp
index 6a7da02..c7489da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp
@@ -17,22 +17,21 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
-
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
-
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+///////////////////////////////////////
+// Step 4: Insert after index created
+
+USE MyDataVerse;
+
+UPSERT INTO MyMessageDataset ([
+  {
+      "myMessageId":    3,
+      "myMessageBody":  "xxx cute cat"
+  }
+]);
+
+UPSERT INTO MyMessageDataset ([
+  {
+      "myMessageId":    4,
+      "myMessageBody":  "yyy crazy cat"
+  }
+]);
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp
index 6a7da02..0387e37 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp
@@ -17,22 +17,11 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+///////////////////////////////////////
+// Step 5: Select with ft_config
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+USE MyDataVerse;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+// Without FT Config
+SELECT VALUE myMessage from MyMessageDataset myMessage
+    WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"], {"mode":"all"}) ORDER BY myMessage.myMessageId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm
new file mode 100644
index 0000000..6785ffe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm
@@ -0,0 +1 @@
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm
new file mode 100644
index 0000000..730d14b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm
@@ -0,0 +1,2 @@
+{ "myMessageId": 1, "myMessageBody": "a smart black dog" }
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm
new file mode 100644
index 0000000..2c0e461
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm
@@ -0,0 +1 @@
+{ "myMessageId": 3, "myMessageBody": "xxx cute cat" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm
new file mode 100644
index 0000000..4c485dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm
@@ -0,0 +1,2 @@
+{ "myMessageId": 3, "myMessageBody": "xxx cute cat" }
+{ "myMessageId": 4, "myMessageBody": "yyy crazy cat" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm
new file mode 100644
index 0000000..522e0ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "MyDataVerse", "FullTextConfigName": "my_first_stopword_config", "Tokenizer": "WORD", "FullTextFilterPipeline": [ "my_first_stopword_filter" ] }
+{ "DataverseName": "MyDataVerse", "FullTextConfigName": "my_second_stopword_config", "Tokenizer": "WORD", "FullTextFilterPipeline": [ "my_second_stopword_filter" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm
new file mode 100644
index 0000000..f117d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "MyDataVerse", "FullTextFilterName": "my_first_stopword_filter", "FullTextFilterType": "Stopwords", "StopwordList": [ "a", "an", "the" ] }
+{ "DataverseName": "MyDataVerse", "FullTextFilterName": "my_second_stopword_filter", "FullTextFilterType": "Stopwords", "StopwordList": [ "xxx", "yyy", "zzz" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
new file mode 100644
index 0000000..ba9055b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
@@ -0,0 +1,4 @@
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "MyMessageDataset", "IndexStructure": "BTREE", "SearchKey": [ [ "myMessageId" ] ], "IsPrimary": true, "Timestamp": "Sat Nov 14 21:16:57 PST 2020", "PendingOp": 0 }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_0", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0 }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_1", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0, "FullTextConfig": "my_first_stopword_config" }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_2", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0, "FullTextConfig": "my_second_stopword_config" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm
new file mode 100644
index 0000000..6785ffe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm
@@ -0,0 +1 @@
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index df8de8d..d249887 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5481,6 +5481,11 @@
         <output-dir compare="Text">fulltext-index-large-data</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="stopwords-full-text-filter-1">
+        <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="global-aggregate">
     <test-case FilePath="global-aggregate">
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
index c85d091..79f13bf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
@@ -155,6 +155,66 @@ public interface IMetadataLockManager {
             throws AlgebricksException;
 
     /**
+     * Acquire read lock on the full-text config
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @param fullTextConfigName
+     *            the name of the full-text config in the given dataverse
+     * @throws AlgebricksException
+     *             if lock couldn't be acquired
+     */
+    void acquireFullTextConfigReadLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+            throws AlgebricksException;
+
+    /**
+     * Acquire write lock on the full-text config
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @param fullTextConfigName
+     *            the name of the full-text config in the given dataverse
+     * @throws AlgebricksException
+     *             if lock couldn't be acquired
+     */
+    void acquireFullTextConfigWriteLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+            throws AlgebricksException;
+
+    /**
+     * Acquire read lock on the full-text filter
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @param fullTextFilterName
+     *            the name of the full-text filter in the given dataverse
+     * @throws AlgebricksException
+     *             if lock couldn't be acquired
+     */
+    void acquireFullTextFilterReadLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+            throws AlgebricksException;
+
+    /**
+     * Acquire write lock on the full-text filter
+     *
+     * @param locks
+     *            the lock list to add the new lock to
+     * @param dataverseName
+     *            the dataverse name
+     * @param fullTextFilterName
+     *            the name of the full-text filter in the given dataverse
+     * @throws AlgebricksException
+     *             if lock couldn't be acquired
+     */
+    void acquireFullTextFilterWriteLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+            throws AlgebricksException;
+
+    /**
      * Acquire read lock on the library
      *
      * @param locks
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 0efddeb..b7b8b66 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -221,6 +221,12 @@ public class ErrorCode {
     public static final int UNSUPPORTED_GBY_OBY_SELECT_COMBO = 1129;
     public static final int ILLEGAL_RIGHT_OUTER_JOIN = 1130;
     public static final int SYNONYM_EXISTS = 1131;
+    public static final int FULL_TEXT_CONFIG_NOT_FOUND = 1132;
+    public static final int FULL_TEXT_FILTER_NOT_FOUND = 1133;
+    public static final int FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED = 1134;
+    public static final int FULL_TEXT_CONFIG_ALREADY_EXISTS = 1135;
+    public static final int FULL_TEXT_FILTER_ALREADY_EXISTS = 1136;
+
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
     public static final int UTIL_DATAFLOW_UTILS_TUPLE_TOO_LARGE = 3002;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index b3a7f54..28eb553 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -23,6 +23,8 @@ import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
+import com.google.common.collect.ImmutableList;
+
 public interface IMetadataLockUtil {
 
     // Dataverse helpers
@@ -59,7 +61,7 @@ public interface IMetadataLockUtil {
     // Index helpers
 
     void createIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
-            String datasetName) throws AlgebricksException;
+            String datasetName, String fullTextConfigName) throws AlgebricksException;
 
     void dropIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException;
@@ -72,7 +74,7 @@ public interface IMetadataLockUtil {
     void dropTypeBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName, String typeName)
             throws AlgebricksException;
 
-    // Function helpers
+    // Library helpers
 
     void createLibraryBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String libraryName) throws AlgebricksException;
@@ -88,6 +90,22 @@ public interface IMetadataLockUtil {
     void dropFunctionBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String functionName) throws AlgebricksException;
 
+    // Full-text filter helpers
+
+    void createFullTextFilterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String fullTextFilterName) throws AlgebricksException;
+
+    void dropFullTextFilterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String fullTextFilterName) throws AlgebricksException;
+
+    // Full-text config helpers
+
+    void createFullTextConfigBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String fullTextConfigName, ImmutableList<String> fullTextFilterNames) throws AlgebricksException;
+
+    void dropFullTextConfigBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String fullTextFilterName) throws AlgebricksException;
+
     // Adapter helpers
 
     void createAdapterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
@@ -129,4 +147,4 @@ public interface IMetadataLockUtil {
 
     void disconnectFeedBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName, String feedName) throws AlgebricksException;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 1a9afd3..9c8bf75 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -218,6 +218,11 @@
 1129 = Cannot compile SELECT variable.* with GROUP BY GROUPING SETS/ROLLUP/CUBE followed by ORDER BY/LIMIT
 1130 = Illegal use of RIGHT OUTER JOIN
 1131 = A synonym with this name %1$s already exists
+1132 = Full-text config %1$s not found
+1133 = Full-text filter %1$s not found
+1134 = Default full-text config with a name of null cannot be deleted or created
+1135 = Full-text config %1$s already exists
+1136 = Full-text filter %1$s already exists
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-fuzzyjoin/pom.xml b/asterixdb/asterix-fuzzyjoin/pom.xml
index 5d8884a..860c293 100644
--- a/asterixdb/asterix-fuzzyjoin/pom.xml
+++ b/asterixdb/asterix-fuzzyjoin/pom.xml
@@ -125,6 +125,10 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 1e3e68f..182fc08 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -68,7 +68,11 @@ public interface Statement extends ILangExpression {
         WRITE,
         CREATE_INDEX,
         CREATE_DATAVERSE,
+        CREATE_FULL_TEXT_FILTER,
+        CREATE_FULL_TEXT_CONFIG,
         INDEX_DROP,
+        FULL_TEXT_FILTER_DROP,
+        FULL_TEXT_CONFIG_DROP,
         CREATE_FEED,
         DROP_FEED,
         START_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java
new file mode 100644
index 0000000..2d304be
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Iterator;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.FullTextUtil;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+public class CreateFullTextConfigStatement extends AbstractStatement {
+
+    private final DataverseName dataverseName;
+    private final String configName;
+    private final boolean ifNotExists;
+    private final AdmObjectNode configNode;
+
+    public static final String FIELD_NAME_TOKENIZER = "tokenizer";
+    public static final String FIELD_NAME_FILTER_PIPELINE = "filterPipeline";
+
+    public CreateFullTextConfigStatement(DataverseName dataverseName, String configName, boolean ifNotExists,
+            RecordConstructor expr) throws CompilationException {
+        this.dataverseName = dataverseName;
+        this.configName = configName;
+        this.ifNotExists = ifNotExists;
+        this.configNode = FullTextUtil.validateAndGetConfigNode(expr);
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getConfigName() {
+        return configName;
+    }
+
+    public boolean getIfNotExists() {
+        return ifNotExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CREATE_FULL_TEXT_CONFIG;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+
+    public TokenizerCategory getTokenizerCategory() throws HyracksDataException {
+        String tokenizerCategoryStr = configNode.getString(FIELD_NAME_TOKENIZER);
+        TokenizerCategory tokenizerCategory = TokenizerCategory.getEnumIgnoreCase(tokenizerCategoryStr);
+
+        return tokenizerCategory;
+    }
+
+    public ImmutableList<String> getFilterNames() {
+        AdmArrayNode arrayNode = (AdmArrayNode) configNode.get(FIELD_NAME_FILTER_PIPELINE);
+        ImmutableList.Builder<String> filterNamesBuilder = ImmutableList.builder();
+
+        Iterator<IAdmNode> iterator = arrayNode.iterator();
+        while (iterator.hasNext()) {
+            filterNamesBuilder.add(((AdmStringNode) iterator.next()).get());
+        }
+
+        return filterNamesBuilder.build();
+    }
+
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java
new file mode 100644
index 0000000..20427fd
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java
@@ -0,0 +1,105 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Iterator;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.FullTextUtil;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.google.common.collect.ImmutableList;
+
+public class CreateFullTextFilterStatement extends AbstractStatement {
+
+    private final DataverseName dataverseName;
+    private final String filterName;
+    private final boolean ifNotExists;
+    private final AdmObjectNode filterNode;
+
+    public static final String FIELD_NAME_TYPE = "type";
+    public static final String FIELD_NAME_STOPWORDS_LIST = "stopwordsList";
+    public static final String FIELD_TYPE_STOPWORDS = "stopwords";
+
+    public CreateFullTextFilterStatement(DataverseName dataverseName, String filterName, boolean ifNotExists,
+            RecordConstructor expr) throws CompilationException {
+        this.dataverseName = dataverseName;
+        this.filterName = filterName;
+        this.ifNotExists = ifNotExists;
+        this.filterNode = FullTextUtil.getFilterNode(expr);
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getFilterName() {
+        return filterName;
+    }
+
+    public boolean getIfNotExists() {
+        return this.ifNotExists;
+    }
+
+    public String getFilterType() throws HyracksDataException {
+        return filterNode.getString(FIELD_NAME_TYPE);
+    }
+
+    public ImmutableList<String> getStopwordsList() throws CompilationException {
+        ImmutableList.Builder listBuiler = ImmutableList.<String> builder();
+        AdmArrayNode arrayNode = (AdmArrayNode) filterNode.get(FIELD_NAME_STOPWORDS_LIST);
+
+        Iterator<IAdmNode> iterator = arrayNode.iterator();
+        while (iterator.hasNext()) {
+            IAdmNode node = iterator.next();
+            if (!(node instanceof AdmStringNode)) {
+                throw new CompilationException(ErrorCode.PARSE_ERROR, getSourceLocation(),
+                        "error when parsing stopwords list");
+            }
+
+            listBuiler.add(((AdmStringNode) node).get());
+        }
+
+        return listBuiler.build();
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CREATE_FULL_TEXT_FILTER;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 05fc055..55d0610 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -44,6 +44,8 @@ public class CreateIndexStatement extends AbstractStatement {
 
     // Specific to NGram indexes.
     private int gramLength = -1;
+    // Specific to FullText indexes.
+    private String fullTextConfigName;
 
     public CreateIndexStatement() {
     }
@@ -56,6 +58,15 @@ public class CreateIndexStatement extends AbstractStatement {
         return gramLength;
     }
 
+    public void setFullTextConfigName(String fullTextConfigName) {
+        this.fullTextConfigName = fullTextConfigName;
+        return;
+    }
+
+    public String getFullTextConfigName() {
+        return fullTextConfigName;
+    }
+
     public Identifier getIndexName() {
         return indexName;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java
new file mode 100644
index 0000000..ad84571
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class FullTextConfigDropStatement extends AbstractStatement {
+
+    private final DataverseName dataverseName;
+    private final String configName;
+    private final boolean ifExists;
+
+    public FullTextConfigDropStatement(DataverseName dataverseName, String configName, boolean ifExists) {
+        this.dataverseName = dataverseName;
+        this.configName = configName;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.FULL_TEXT_CONFIG_DROP;
+    }
+
+    public boolean getIfExists() {
+        return ifExists;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getConfigName() {
+        return configName;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java
new file mode 100644
index 0000000..9debca0
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class FullTextFilterDropStatement extends AbstractStatement {
+
+    private final DataverseName dataverseName;
+    private final String filterName;
+    private final boolean ifExists;
+
+    public FullTextFilterDropStatement(DataverseName dataverseName, String filterName, boolean ifExists) {
+        this.dataverseName = dataverseName;
+        this.filterName = filterName;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.FULL_TEXT_FILTER_DROP;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getFilterName() {
+        return filterName;
+    }
+
+    public boolean getIfExists() {
+        return ifExists;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java
new file mode 100644
index 0000000..c071dc9
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.util;
+
+import static org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement.FIELD_NAME_FILTER_PIPELINE;
+import static org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement.FIELD_NAME_TOKENIZER;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class FullTextUtil {
+
+    private FullTextUtil() {
+    }
+
+    //--------------------------------------- Full-text config --------------------------------------//
+
+    // Example of full-text config create statement
+    // CREATE FULLTEXT CONFIG my_second_stopword_config IF NOT EXISTS AS {
+    //     "Tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+    //     "FilterPipeline": ["my_second_stopword_filter"]
+    // };
+    private static ARecordType getFullTextConfigRecordType() {
+        final String[] fieldNames = { FIELD_NAME_TOKENIZER, FIELD_NAME_FILTER_PIPELINE };
+        final IAType[] fieldTypes = { BuiltinType.ASTRING, new AOrderedListType(BuiltinType.ASTRING, null) };
+        return new ARecordType("fullTextConfigRecordType", fieldNames, fieldTypes, true);
+    }
+
+    private static final ARecordType FULL_TEXT_CONFIG_RECORD_TYPE = getFullTextConfigRecordType();
+
+    public static AdmObjectNode validateAndGetConfigNode(RecordConstructor recordConstructor)
+            throws CompilationException {
+        final ConfigurationTypeValidator validator = new ConfigurationTypeValidator();
+        final AdmObjectNode node = ExpressionUtils.toNode(recordConstructor);
+        validator.validateType(FULL_TEXT_CONFIG_RECORD_TYPE, node);
+        return node;
+    }
+
+    //--------------------------------------- Full-text filter --------------------------------------//
+
+    // Example of full-text filter create statement
+    // Note that only the type field is a must, and other fields is filter-type-specific
+    //
+    // CREATE FULLTEXT FILTER my_stopword_filter IF NOT EXISTS AS {
+    //     "Type": "stopwords",
+    //     "StopwordsList": ["xxx", "yyy", "zzz"]
+    // };
+
+    public static AdmObjectNode getFilterNode(RecordConstructor recordConstructor) throws CompilationException {
+        // Skip validation here because the current validator only supports CLOSED record validate
+        // while the FULL_TEXT_FILTER_RECORD_TYPE is open and specific to the filter types,
+        // e.g. stopwords filter and stemmer filter may have different fields
+        final AdmObjectNode node = ExpressionUtils.toNode(recordConstructor);
+        return node;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0474c0e..ac08705 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -70,6 +70,8 @@ import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -83,6 +85,8 @@ import org.apache.asterix.lang.common.statement.DropDatasetStatement;
 import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
 import org.apache.asterix.lang.common.statement.FeedDropStatement;
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -686,6 +690,20 @@ public abstract class FormatPrintVisitor implements ILangVisitor<Void, Integer>
     }
 
     @Override
+    public Void visit(CreateFullTextFilterStatement cis, Integer step) throws CompilationException {
+        out.print(skip(step) + "create fulltext filter " + cis.getFilterName());
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(CreateFullTextConfigStatement cis, Integer step) throws CompilationException {
+        out.print(skip(step) + "create fulltext config " + cis.getConfigName());
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
     public Void visit(IndexDropStatement del, Integer step) throws CompilationException {
         out.print(skip(step) + "drop index ");
         out.print(generateFullName(del.getDataverseName(), del.getDatasetName()));
@@ -719,6 +737,20 @@ public abstract class FormatPrintVisitor implements ILangVisitor<Void, Integer>
     }
 
     @Override
+    public Void visit(FullTextFilterDropStatement del, Integer step) throws CompilationException {
+        out.print(skip(step) + "drop fulltext filter " + del.getFilterName());
+        out.println(generateIfExists(del.getIfExists()) + SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(FullTextConfigDropStatement del, Integer step) throws CompilationException {
+        out.print(skip(step) + "drop fulltext config " + del.getConfigName());
+        out.println(generateIfExists(del.getIfExists()) + SEMICOLON);
+        return null;
+    }
+
+    @Override
     public Void visit(ConnectFeedStatement connectFeedStmt, Integer step) throws CompilationException {
         out.print(skip(step) + "connect " + FEED);
         out.print(generateFullName(connectFeedStmt.getDataverseName(), new Identifier(connectFeedStmt.getFeedName())));
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 9f4571d..0b4f56b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -31,6 +31,8 @@ import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -43,6 +45,8 @@ import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
 import org.apache.asterix.lang.common.statement.DropDatasetStatement;
 import org.apache.asterix.lang.common.statement.FeedDropStatement;
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -184,6 +188,26 @@ public abstract class AbstractQueryExpressionVisitor<R, T> implements ILangVisit
     }
 
     @Override
+    public R visit(CreateFullTextFilterStatement del, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(FullTextFilterDropStatement del, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(CreateFullTextConfigStatement del, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(FullTextConfigDropStatement del, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
     public R visit(DisconnectFeedStatement del, T arg) throws CompilationException {
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index 2957509..e36ec73 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -48,6 +48,8 @@ import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -60,6 +62,8 @@ import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
 import org.apache.asterix.lang.common.statement.DropDatasetStatement;
 import org.apache.asterix.lang.common.statement.FeedDropStatement;
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -144,8 +148,16 @@ public interface ILangVisitor<R, T> {
 
     R visit(CreateDataverseStatement del, T arg) throws CompilationException;
 
+    R visit(CreateFullTextFilterStatement cis, T arg) throws CompilationException;
+
+    R visit(CreateFullTextConfigStatement cis, T arg) throws CompilationException;
+
     R visit(IndexDropStatement del, T arg) throws CompilationException;
 
+    R visit(FullTextFilterDropStatement del, T arg) throws CompilationException;
+
+    R visit(FullTextConfigDropStatement del, T arg) throws CompilationException;
+
     R visit(NodeGroupDropStatement del, T arg) throws CompilationException;
 
     R visit(DataverseDropStatement del, T arg) throws CompilationException;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index a4995da..def5742 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -131,6 +131,8 @@ import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -143,6 +145,8 @@ import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
 import org.apache.asterix.lang.common.statement.LoadStatement;
@@ -237,6 +241,8 @@ class SQLPPParser extends ScopeChecker implements IParser {
     private static final String UNBOUNDED = "UNBOUNDED";
     private static final String REPLACE = "REPLACE";
     private static final String RETURNS = "RETURNS";
+    private static final String CONFIG = "CONFIG";
+
 
     private static final String INT_TYPE_NAME = "int";
     private static final String UDF_VARARGS_PARAM_NAME = "args"; // Note: this value is stored in the function metadata
@@ -257,10 +263,12 @@ class SQLPPParser extends ScopeChecker implements IParser {
     private static class IndexParams {
       public IndexType type;
       public int gramLength;
+      public String fullTextConfig;
 
-      public IndexParams(IndexType type, int gramLength) {
+      public IndexParams(IndexType type, int gramLength, String fullTextConfig) {
         this.type = type;
         this.gramLength = gramLength;
+        this.fullTextConfig = fullTextConfig;
       }
     };
 
@@ -693,6 +701,7 @@ Statement CreateStatement() throws ParseException:
     | stmt = CreateSynonymStatement(startToken)
     | stmt = CreateFeedStatement(startToken)
     | stmt = CreateFeedPolicyStatement(startToken)
+    | stmt = CreateFullTextStatement(startToken)
   )
   {
     return stmt;
@@ -1027,6 +1036,7 @@ CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseExcept
     if (indexType != null) {
       stmt.setIndexType(indexType.type);
       stmt.setGramLength(indexType.gramLength);
+      stmt.setFullTextConfigName(indexType.fullTextConfig);
     }
     stmt.setEnforced(enforced);
     return addSourceLocation(stmt, startStmtToken);
@@ -1081,6 +1091,7 @@ IndexParams IndexType() throws ParseException:
 {
   IndexType type = null;
   int gramLength = 0;
+  String fullTextConfig = null;
 }
 {
   (<BTREE>
@@ -1095,10 +1106,18 @@ IndexParams IndexType() throws ParseException:
     {
       type = IndexType.LENGTH_PARTITIONED_WORD_INVIX;
     }
-  |<FULLTEXT>
+  | <FULLTEXT>
     {
       type = IndexType.SINGLE_PARTITION_WORD_INVIX;
     }
+    // For now we don't allow inverted index creation using a full-text config in another data verse.
+    // We may want to support corss-dataverse full-text config access later
+    // If so, replace the Identifier() with QualifiedName() to get the dataverse name
+    ( <USING> Identifier()
+      {
+        fullTextConfig = token.image;
+      }
+    )?
   | <NGRAM> <LEFTPAREN> <INTEGER_LITERAL>
     {
       type = IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
@@ -1106,7 +1125,7 @@ IndexParams IndexType() throws ParseException:
     }
   <RIGHTPAREN>)
     {
-      return new IndexParams(type, gramLength);
+      return new IndexParams(type, gramLength, fullTextConfig);
     }
 }
 
@@ -1418,6 +1437,69 @@ CreateFeedPolicyStatement FeedPolicySpecification(Token startStmtToken) throws P
   }
 }
 
+Statement CreateFullTextStatement(Token startStmtToken) throws ParseException:
+{
+  Statement stmt = null;
+}
+{
+  (
+    <FULLTEXT>
+    (
+      <FILTER> stmt = CreateFullTextFilterSpec(startStmtToken)
+      | (<IDENTIFIER> { expectToken(CONFIG); } stmt = CreateFullTextConfigSpec(startStmtToken))
+    )
+  )
+  {
+    return stmt;
+  }
+}
+
+CreateFullTextFilterStatement CreateFullTextFilterSpec(Token startStmtToken) throws ParseException:
+{
+  CreateFullTextFilterStatement stmt = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+  boolean ifNotExists = false;
+  RecordConstructor expr = null;
+}
+{
+  (
+    nameComponents = QualifiedName() ifNotExists = IfNotExists()
+    <AS>
+    expr = RecordConstructor()
+  )
+  {
+    try {
+      stmt = new CreateFullTextFilterStatement(nameComponents.first, nameComponents.second.getValue(), ifNotExists, expr);
+      return addSourceLocation(stmt, startStmtToken);
+    } catch (CompilationException e) {
+       throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
+    }
+  }
+}
+
+CreateFullTextConfigStatement CreateFullTextConfigSpec(Token startStmtToken) throws ParseException:
+{
+  CreateFullTextConfigStatement stmt = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+  boolean ifNotExists = false;
+  RecordConstructor expr = null;
+}
+{
+  (
+    nameComponents = QualifiedName() ifNotExists = IfNotExists()
+    <AS>
+    expr = RecordConstructor()
+  )
+  {
+    try {
+      stmt = new CreateFullTextConfigStatement(nameComponents.first, nameComponents.second.getValue(), ifNotExists, expr);
+      return addSourceLocation(stmt, startStmtToken);
+    } catch (CompilationException e) {
+       throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
+    }
+  }
+}
+
 CreateSynonymStatement CreateSynonymStatement(Token startStmtToken) throws ParseException:
 {
   CreateSynonymStatement stmt = null;
@@ -1569,6 +1651,7 @@ Statement DropStatement() throws ParseException:
     | stmt = DropFeedStatement(startToken)
     | stmt = DropFeedPolicyStatement(startToken)
     | stmt = DropSynonymStatement(startToken)
+    | stmt = DropFullTextStatement(startToken)
   )
   {
     return stmt;
@@ -1623,6 +1706,50 @@ IndexDropStatement DropIndexSpecification(Token startStmtToken) throws ParseExce
   }
 }
 
+Statement DropFullTextStatement(Token startStmtToken) throws ParseException:
+{
+  Statement stmt = null;
+}
+{
+  <FULLTEXT>
+  (
+    <FILTER> stmt = DropFullTextFilterSpec(startStmtToken)
+     | (<IDENTIFIER> { expectToken(CONFIG); } stmt = DropFullTextConfigSpec(startStmtToken))
+  )
+  {
+    return stmt;
+  }
+}
+
+FullTextFilterDropStatement DropFullTextFilterSpec(Token startStmtToken) throws ParseException:
+{
+  FullTextFilterDropStatement stmt = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+  boolean ifExists = false;
+}
+{
+  nameComponents = QualifiedName() ifExists = IfExists()
+  {
+    stmt = new FullTextFilterDropStatement(nameComponents.first, nameComponents.second.getValue(), ifExists);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
+FullTextConfigDropStatement DropFullTextConfigSpec(Token startStmtToken) throws ParseException:
+{
+  FullTextConfigDropStatement stmt = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+  boolean ifExists = false;
+}
+{
+  nameComponents = QualifiedName() ifExists = IfExists()
+  {
+    stmt = new FullTextConfigDropStatement(nameComponents.first, nameComponents.second.getValue(), ifExists);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
+
 NodeGroupDropStatement DropNodeGroupStatement(Token startStmtToken) throws ParseException:
 {
   NodeGroupDropStatement stmt = null;
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 1e2bf53..ba943a6 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -178,7 +178,11 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs-client</artifactId>
     </dependency>
-      <dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-ipc</artifactId>
     </dependency>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index 116e55b..6602666 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -37,12 +37,15 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
 
 /**
  * Caches metadata entities such that the MetadataManager does not have to
@@ -79,6 +82,10 @@ public class MetadataCache {
     protected final Map<DataverseName, Map<String, FeedConnection>> feedConnections = new HashMap<>();
     // Key is synonym dataverse. Key of value map is the synonym name
     protected final Map<DataverseName, Map<String, Synonym>> synonyms = new HashMap<>();
+    // Key is DataverseName. Key of value map is the full-text filter name
+    protected final Map<DataverseName, Map<String, FullTextFilterMetadataEntity>> fullTextFilters = new HashMap<>();
+    // Key is DataverseName. Key of value map is the full-text config name
+    protected final Map<DataverseName, Map<String, FullTextConfigMetadataEntity>> fullTextConfigs = new HashMap<>();
 
     // Atomically executes all metadata operations in ctx's log.
     public void commit(MetadataTransactionContext ctx) {
@@ -113,20 +120,26 @@ public class MetadataCache {
                     synchronized (indexes) {
                         synchronized (datatypes) {
                             synchronized (functions) {
-                                synchronized (adapters) {
-                                    synchronized (libraries) {
-                                        synchronized (compactionPolicies) {
-                                            synchronized (synonyms) {
-                                                dataverses.clear();
-                                                nodeGroups.clear();
-                                                datasets.clear();
-                                                indexes.clear();
-                                                datatypes.clear();
-                                                functions.clear();
-                                                adapters.clear();
-                                                libraries.clear();
-                                                compactionPolicies.clear();
-                                                synonyms.clear();
+                                synchronized (fullTextConfigs) {
+                                    synchronized (fullTextFilters) {
+                                        synchronized (adapters) {
+                                            synchronized (libraries) {
+                                                synchronized (compactionPolicies) {
+                                                    synchronized (synonyms) {
+                                                        dataverses.clear();
+                                                        nodeGroups.clear();
+                                                        datasets.clear();
+                                                        indexes.clear();
+                                                        datatypes.clear();
+                                                        functions.clear();
+                                                        fullTextConfigs.clear();
+                                                        fullTextFilters.clear();
+                                                        adapters.clear();
+                                                        libraries.clear();
+                                                        compactionPolicies.clear();
+                                                        synonyms.clear();
+                                                    }
+                                                }
                                             }
                                         }
                                     }
@@ -237,30 +250,37 @@ public class MetadataCache {
                 synchronized (indexes) {
                     synchronized (datatypes) {
                         synchronized (functions) {
-                            synchronized (adapters) {
-                                synchronized (libraries) {
-                                    synchronized (feeds) {
-                                        synchronized (compactionPolicies) {
-                                            synchronized (synonyms) {
-                                                datasets.remove(dataverse.getDataverseName());
-                                                indexes.remove(dataverse.getDataverseName());
-                                                datatypes.remove(dataverse.getDataverseName());
-                                                adapters.remove(dataverse.getDataverseName());
-                                                compactionPolicies.remove(dataverse.getDataverseName());
-                                                List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<>();
-                                                for (FunctionSignature signature : functions.keySet()) {
-                                                    if (signature.getDataverseName()
-                                                            .equals(dataverse.getDataverseName())) {
-                                                        markedFunctionsForRemoval.add(signature);
+                            synchronized (fullTextConfigs) {
+                                synchronized (fullTextFilters) {
+                                    synchronized (adapters) {
+                                        synchronized (libraries) {
+                                            synchronized (feeds) {
+                                                synchronized (compactionPolicies) {
+                                                    synchronized (synonyms) {
+                                                        datasets.remove(dataverse.getDataverseName());
+                                                        indexes.remove(dataverse.getDataverseName());
+                                                        datatypes.remove(dataverse.getDataverseName());
+                                                        adapters.remove(dataverse.getDataverseName());
+                                                        compactionPolicies.remove(dataverse.getDataverseName());
+                                                        List<FunctionSignature> markedFunctionsForRemoval =
+                                                                new ArrayList<>();
+                                                        for (FunctionSignature signature : functions.keySet()) {
+                                                            if (signature.getDataverseName()
+                                                                    .equals(dataverse.getDataverseName())) {
+                                                                markedFunctionsForRemoval.add(signature);
+                                                            }
+                                                        }
+                                                        for (FunctionSignature signature : markedFunctionsForRemoval) {
+                                                            functions.remove(signature);
+                                                        }
+                                                        fullTextConfigs.remove(dataverse.getDataverseName());
+                                                        fullTextFilters.remove(dataverse.getDataverseName());
+                                                        libraries.remove(dataverse.getDataverseName());
+                                                        feeds.remove(dataverse.getDataverseName());
+                                                        synonyms.remove(dataverse.getDataverseName());
+                                                        return dataverses.remove(dataverse.getDataverseName());
                                                     }
                                                 }
-                                                for (FunctionSignature signature : markedFunctionsForRemoval) {
-                                                    functions.remove(signature);
-                                                }
-                                                libraries.remove(dataverse.getDataverseName());
-                                                feeds.remove(dataverse.getDataverseName());
-                                                synonyms.remove(dataverse.getDataverseName());
-                                                return dataverses.remove(dataverse.getDataverseName());
                                             }
                                         }
                                     }
@@ -376,6 +396,26 @@ public class MetadataCache {
         }
     }
 
+    public FullTextConfigMetadataEntity getFullTextConfig(DataverseName dataverseName, String configName) {
+        synchronized (fullTextConfigs) {
+            Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.get(configName);
+        }
+    }
+
+    public FullTextFilterMetadataEntity getFullTextFilter(DataverseName dataverseName, String filterName) {
+        synchronized (fullTextFilters) {
+            Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.get(filterName);
+        }
+    }
+
     public List<Dataset> getDataverseDatasets(DataverseName dataverseName) {
         synchronized (datasets) {
             Map<String, Dataset> m = datasets.get(dataverseName);
@@ -436,6 +476,65 @@ public class MetadataCache {
         }
     }
 
+    public FullTextFilterMetadataEntity addFullTextFilterIfNotExists(FullTextFilterMetadataEntity filter) {
+        DataverseName dataverseName = filter.getFullTextFilter().getDataverseName();
+        String filterName = filter.getFullTextFilter().getName();
+        synchronized (fullTextFilters) {
+            Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+            if (m == null) {
+                m = new HashMap<>();
+                fullTextFilters.put(dataverseName, m);
+            }
+            if (!m.containsKey(filterName)) {
+                return m.put(filterName, filter);
+            }
+            return null;
+        }
+    }
+
+    public FullTextFilterMetadataEntity dropFullTextFilter(FullTextFilterMetadataEntity filterMetadataEntity) {
+        DataverseName dataverseName = filterMetadataEntity.getFullTextFilter().getDataverseName();
+        String filterName = filterMetadataEntity.getFullTextFilter().getName();
+        synchronized (fullTextFilters) {
+            Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.remove(filterName);
+        }
+    }
+
+    public FullTextConfigMetadataEntity addFullTextConfigIfNotExists(
+            FullTextConfigMetadataEntity configMetadataEntity) {
+        FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+        DataverseName dataverseName = config.getDataverseName();
+        String configName = config.getName();
+        synchronized (fullTextConfigs) {
+            Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+            if (m == null) {
+                m = new HashMap<>();
+                fullTextConfigs.put(dataverseName, m);
+            }
+            if (!m.containsKey(configName)) {
+                return m.put(configName, configMetadataEntity);
+            }
+            return null;
+        }
+    }
+
+    public FullTextConfigMetadataEntity dropFullTextConfig(FullTextConfigMetadataEntity configMetadataEntity) {
+        FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+        DataverseName dataverseName = config.getDataverseName();
+        String configName = config.getName();
+        synchronized (fullTextConfigs) {
+            Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.remove(configName);
+        }
+    }
+
     public Object addFeedPolicyIfNotExists(FeedPolicyEntity feedPolicy) {
         synchronized (feedPolicies) {
             Map<String, FeedPolicyEntity> p = feedPolicies.get(feedPolicy.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 268a9c0..798c604 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
@@ -49,6 +50,8 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
@@ -62,6 +65,8 @@ import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.google.common.base.Strings;
+
 /**
  * Provides access to Asterix metadata via remote methods to the metadata node.
  * This metadata manager maintains a local cache of metadata Java objects
@@ -579,7 +584,7 @@ public abstract class MetadataManager implements IMetadataManager {
         // requested function itself (but the function is still uncommitted).
         Function function = ctx.getFunction(functionSignature);
         if (function != null) {
-            // Don't add this dataverse to the cache, since it is still
+            // Don't add this function to the cache, since it is still
             // uncommitted.
             return function;
         }
@@ -622,6 +627,141 @@ public abstract class MetadataManager implements IMetadataManager {
     }
 
     @Override
+    public void addFullTextFilter(MetadataTransactionContext mdTxnCtx, FullTextFilterMetadataEntity filter)
+            throws AlgebricksException {
+        try {
+            metadataNode.addFullTextFilter(mdTxnCtx.getTxnId(), filter);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        mdTxnCtx.addFullTextFilter(filter);
+    }
+
+    @Override
+    public void dropFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String filterName)
+            throws AlgebricksException {
+        try {
+            metadataNode.dropFullTextFilter(mdTxnCtx.getTxnId(), dataverseName, filterName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        mdTxnCtx.dropFullTextFilter(dataverseName, filterName);
+    }
+
+    @Override
+    public FullTextFilterMetadataEntity getFullTextFilter(MetadataTransactionContext ctx, DataverseName dataverseName,
+            String filterName) throws AlgebricksException {
+        // First look in the context to see if this transaction created the
+        // requested full-text filter itself (but the full-text filter is still uncommitted).
+        FullTextFilterMetadataEntity filter = ctx.getFullTextFilter(dataverseName, filterName);
+        if (filter != null) {
+            // Don't add this filter to the cache, since it is still
+            // uncommitted.
+            return filter;
+        }
+
+        if (ctx.fullTextFilterIsDropped(dataverseName, filterName)) {
+            // Filter has been dropped by this transaction but could still be
+            // in the cache.
+            return null;
+        }
+
+        if (ctx.getDataverse(dataverseName) != null) {
+            // This transaction has dropped and subsequently created the same
+            // dataverse.
+            return null;
+        }
+
+        filter = cache.getFullTextFilter(dataverseName, filterName);
+        if (filter != null) {
+            // filter is already in the cache, don't add it again.
+            return filter;
+        }
+
+        try {
+            filter = metadataNode.getFullTextFilter(ctx.getTxnId(), dataverseName, filterName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        // We fetched the filter from the MetadataNode. Add it to the cache
+        // when this transaction commits.
+        if (filter != null) {
+            ctx.addFullTextFilter(filter);
+        }
+        return filter;
+    }
+
+    @Override
+    public void addFullTextConfig(MetadataTransactionContext mdTxnCtx,
+            FullTextConfigMetadataEntity configMetadataEntity) throws AlgebricksException {
+        if (Strings.isNullOrEmpty(configMetadataEntity.getFullTextConfig().getName())) {
+            throw new AsterixException(ErrorCode.FULL_TEXT_CONFIG_ALREADY_EXISTS);
+        }
+
+        try {
+            metadataNode.addFullTextConfig(mdTxnCtx.getTxnId(), configMetadataEntity);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        mdTxnCtx.addFullTextConfig(configMetadataEntity);
+    }
+
+    @Override
+    public FullTextConfigMetadataEntity getFullTextConfig(MetadataTransactionContext ctx, DataverseName dataverseName,
+            String configName) throws AlgebricksException {
+        // First look in the context to see if this transaction created the
+        // requested full-text config itself (but the full-text config is still uncommitted).
+        FullTextConfigMetadataEntity configMetadataEntity = ctx.getFullTextConfig(dataverseName, configName);
+        if (configMetadataEntity != null) {
+            // Don't add this config to the cache, since it is still
+            // uncommitted.
+            return configMetadataEntity;
+        }
+
+        if (ctx.fullTextConfigIsDropped(dataverseName, configName)) {
+            // config has been dropped by this transaction but could still be
+            // in the cache.
+            return null;
+        }
+
+        if (ctx.getDataverse(dataverseName) != null) {
+            // This transaction has dropped and subsequently created the same
+            // dataverse.
+            return null;
+        }
+
+        configMetadataEntity = cache.getFullTextConfig(dataverseName, configName);
+        if (configMetadataEntity != null) {
+            // config is already in the cache, don't add it again.
+            return configMetadataEntity;
+        }
+
+        try {
+            configMetadataEntity = metadataNode.getFullTextConfig(ctx.getTxnId(), dataverseName, configName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+
+        // We fetched the config from the MetadataNode. Add it to the cache
+        // when this transaction commits.
+        if (configMetadataEntity != null) {
+            ctx.addFullTextConfig(configMetadataEntity);
+        }
+        return configMetadataEntity;
+    }
+
+    @Override
+    public void dropFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String configName)
+            throws AlgebricksException {
+        try {
+            metadataNode.dropFullTextConfig(mdTxnCtx.getTxnId(), dataverseName, configName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        mdTxnCtx.dropFullTextConfig(dataverseName, configName);
+    }
+
+    @Override
     public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicyEntity feedPolicy)
             throws AlgebricksException {
         try {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 88098f8..c078e8b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -19,6 +19,8 @@
 
 package org.apache.asterix.metadata;
 
+import static org.apache.asterix.common.exceptions.ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED;
+
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -31,6 +33,7 @@ import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
@@ -62,6 +65,8 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -78,6 +83,8 @@ import org.apache.asterix.metadata.entitytupletranslators.ExternalFileTupleTrans
 import org.apache.asterix.metadata.entitytupletranslators.FeedConnectionTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.FullTextConfigMetadataEntityTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.FullTextFilterMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
@@ -98,6 +105,7 @@ import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractComplexType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
 import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallback;
@@ -130,6 +138,8 @@ import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.google.common.base.Strings;
+
 public class MetadataNode implements IMetadataNode {
     private static final long serialVersionUID = 1L;
     private static final Logger LOGGER = LogManager.getLogger();
@@ -446,6 +456,135 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
+    @Override
+    public void addFullTextFilter(TxnId txnId, FullTextFilterMetadataEntity filterMetadataEntity)
+            throws RemoteException, AlgebricksException {
+        insertFullTextFilterMetadataEntityToCatalog(txnId, filterMetadataEntity);
+        return;
+    }
+
+    @Override
+    public FullTextFilterMetadataEntity getFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+            throws AlgebricksException {
+        try {
+            FullTextFilterMetadataEntityTupleTranslator translator =
+                    tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+            ITupleReference searchKey = createTuple(dataverseName.getCanonicalForm(), filterName);
+            IValueExtractor<FullTextFilterMetadataEntity> valueExtractor =
+                    new MetadataEntityValueExtractor<>(translator);
+            List<FullTextFilterMetadataEntity> results = new ArrayList<>();
+            searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, searchKey, valueExtractor, results);
+            if (results.isEmpty()) {
+                return null;
+            }
+            return results.get(0);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
+    public void dropFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+            throws AlgebricksException {
+        dropFullTextFilterDescriptor(txnId, dataverseName, filterName, false);
+    }
+
+    private void dropFullTextFilterDescriptor(TxnId txnId, DataverseName dataverseName, String filterName,
+            boolean force) throws AlgebricksException {
+        if (!force) {
+            confirmFullTextFilterCanBeDeleted(txnId, dataverseName, filterName);
+        }
+
+        try {
+            FullTextFilterMetadataEntityTupleTranslator translator =
+                    tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+
+            ITupleReference key = createTuple(dataverseName.getCanonicalForm(), filterName);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, key);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    private void insertFullTextConfigMetadataEntityToCatalog(TxnId txnId, FullTextConfigMetadataEntity config)
+            throws AlgebricksException {
+        try {
+            FullTextConfigMetadataEntityTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+            ITupleReference configTuple = tupleReaderWriter.getTupleFromMetadataEntity(config);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, configTuple);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    private void insertFullTextFilterMetadataEntityToCatalog(TxnId txnId, FullTextFilterMetadataEntity filter)
+            throws AlgebricksException {
+        try {
+            FullTextFilterMetadataEntityTupleTranslator tupleReaderWriter =
+                    tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+            ITupleReference filterTuple = tupleReaderWriter.getTupleFromMetadataEntity(filter);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, filterTuple);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
+    public void addFullTextConfig(TxnId txnId, FullTextConfigMetadataEntity config)
+            throws AlgebricksException, RemoteException {
+        try {
+            insertFullTextConfigMetadataEntityToCatalog(txnId, config);
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e, ErrorCode.ERROR_PROCESSING_TUPLE);
+        }
+    }
+
+    @Override
+    public FullTextConfigMetadataEntity getFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+            throws AlgebricksException {
+        FullTextConfigMetadataEntityTupleTranslator translator =
+                tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+
+        ITupleReference searchKey;
+        List<FullTextConfigMetadataEntity> results = new ArrayList<>();
+        try {
+            searchKey = createTuple(dataverseName.getCanonicalForm(), configName);
+            IValueExtractor<FullTextConfigMetadataEntity> valueExtractor =
+                    new MetadataEntityValueExtractor<>(translator);
+            searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, searchKey, valueExtractor, results);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+
+        if (results.isEmpty()) {
+            return null;
+        }
+
+        FullTextConfigMetadataEntity result = results.get(0);
+        return result;
+    }
+
+    @Override
+    public void dropFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+            throws AlgebricksException {
+        dropFullTextConfigDescriptor(txnId, dataverseName, configName, false);
+    }
+
+    private void dropFullTextConfigDescriptor(TxnId txnId, DataverseName dataverseName, String configName,
+            boolean force) throws AlgebricksException {
+        if (!force) {
+            confirmFullTextConfigCanBeDeleted(txnId, dataverseName, configName);
+        }
+
+        try {
+            ITupleReference key = createTuple(dataverseName.getCanonicalForm(), configName);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, key);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     private void insertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws HyracksDataException {
         modifyMetadataIndex(Operation.INSERT, txnId, metadataIndex, tuple);
@@ -569,6 +708,25 @@ public class MetadataNode implements IMetadataNode {
                 dropDataset(txnId, dataverseName, ds.getDatasetName(), true);
             }
 
+            // Drop full-text configs in this dataverse.
+            // Note that full-text configs are utilized by the index, and we need to always drop index first
+            // and then full-text config
+            List<FullTextConfigMetadataEntity> configMetadataEntities =
+                    getDataverseFullTextConfigs(txnId, dataverseName);
+            for (FullTextConfigMetadataEntity configMetadataEntity : configMetadataEntities) {
+                dropFullTextConfigDescriptor(txnId, dataverseName, configMetadataEntity.getFullTextConfig().getName(),
+                        true);
+            }
+
+            // Drop full-text filters in this dataverse.
+            // Note that full-text filters are utilized by the full-text configs,
+            // and we need to always drop full-text configs first
+            // and then full-text filter
+            List<FullTextFilterMetadataEntity> filters = getDataverseFullTextFilters(txnId, dataverseName);
+            for (FullTextFilterMetadataEntity filter : filters) {
+                dropFullTextFilterDescriptor(txnId, dataverseName, filter.getFullTextFilter().getName(), true);
+            }
+
             // Drop all types in this dataverse.
             // As a side effect, acquires an S lock on the 'datatype' dataset on behalf of txnId.
             List<Datatype> dataverseDatatypes = getDataverseDatatypes(txnId, dataverseName);
@@ -601,7 +759,9 @@ public class MetadataNode implements IMetadataNode {
                 || !getDataverseFunctions(txnId, dataverseName).isEmpty()
                 || !getDataverseFeedPolicies(txnId, dataverseName).isEmpty()
                 || !getDataverseFeeds(txnId, dataverseName).isEmpty()
-                || !getDataverseSynonyms(txnId, dataverseName).isEmpty();
+                || !getDataverseSynonyms(txnId, dataverseName).isEmpty()
+                || !getDataverseFullTextConfigs(txnId, dataverseName).isEmpty()
+                || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty();
     }
 
     @Override
@@ -860,6 +1020,38 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
+    private List<FullTextConfigMetadataEntity> getDataverseFullTextConfigs(TxnId txnId, DataverseName dataverseName)
+            throws AlgebricksException {
+        ITupleReference searchKey = createTuple(dataverseName);
+        FullTextConfigMetadataEntityTupleTranslator tupleReaderWriter =
+                tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+        IValueExtractor<FullTextConfigMetadataEntity> valueExtractor =
+                new MetadataEntityValueExtractor<>(tupleReaderWriter);
+        List<FullTextConfigMetadataEntity> results = new ArrayList<>();
+        try {
+            searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, searchKey, valueExtractor, results);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+        return results;
+    }
+
+    private List<FullTextFilterMetadataEntity> getDataverseFullTextFilters(TxnId txnId, DataverseName dataverseName)
+            throws AlgebricksException {
+        ITupleReference searchKey = createTuple(dataverseName);
+        FullTextFilterMetadataEntityTupleTranslator tupleReaderWriter =
+                tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+        IValueExtractor<FullTextFilterMetadataEntity> valueExtractor =
+                new MetadataEntityValueExtractor<>(tupleReaderWriter);
+        List<FullTextFilterMetadataEntity> results = new ArrayList<>();
+        try {
+            searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, searchKey, valueExtractor, results);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+        return results;
+    }
+
     @Override
     public Dataset getDataset(TxnId txnId, DataverseName dataverseName, String datasetName) throws AlgebricksException {
         try {
@@ -1041,6 +1233,31 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
+    private void confirmFullTextConfigCanBeDeleted(TxnId txnId, DataverseName dataverseNameFullTextConfig,
+            String configName) throws AlgebricksException {
+        if (Strings.isNullOrEmpty(configName)) {
+            throw new MetadataException(FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED);
+        }
+
+        // If any index uses this full-text config, throw an error
+        List<Dataset> datasets = getAllDatasets(txnId);
+        for (Dataset dataset : datasets) {
+            List<Index> indexes = getDatasetIndexes(txnId, dataset.getDataverseName(), dataset.getDatasetName());
+            for (Index index : indexes) {
+                // ToDo: to support index to access full-text config in another dataverse,
+                //   we may need to include the dataverse of the full-text config in the index.getFullTextConfigDataverse()
+                //   and instead of checking index.getDataverseName(), we need to check index.getFullTextConfigDataverse()
+                //   to see if it is the same as the dataverse of the full-text config
+                String indexConfigName = index.getFullTextConfigName();
+                if (index.getDataverseName().equals(dataverseNameFullTextConfig)
+                        && !Strings.isNullOrEmpty(indexConfigName) && indexConfigName.equals(configName)) {
+                    throw new AlgebricksException("Cannot drop full-text config "
+                            + " because it is being used by index " + index.getIndexName());
+                }
+            }
+        }
+    }
+
     private void confirmLibraryCanBeDeleted(TxnId txnId, DataverseName dataverseName, String libraryName)
             throws AlgebricksException {
         confirmLibraryIsUnusedByFunctions(txnId, dataverseName, libraryName);
@@ -1131,6 +1348,22 @@ public class MetadataNode implements IMetadataNode {
         }
     }
 
+    private void confirmFullTextFilterCanBeDeleted(TxnId txnId, DataverseName dataverseName, String fullTextFilterName)
+            throws AlgebricksException {
+        List<FullTextConfigMetadataEntity> configMetadataEntities = getDataverseFullTextConfigs(txnId, dataverseName);
+        for (FullTextConfigMetadataEntity configMetadataEntity : configMetadataEntities) {
+            FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+            for (String filterName : config.getFilterNames()) {
+                if (filterName.equals(fullTextFilterName)) {
+                    throw new AlgebricksException("Cannot drop full-text filter "
+                            + TypeUtil.getFullyQualifiedDisplayName(dataverseName, fullTextFilterName)
+                            + " being used by full-text config "
+                            + TypeUtil.getFullyQualifiedDisplayName(dataverseName, config.getName()));
+                }
+            }
+        }
+    }
+
     private List<String> getNestedComplexDatatypeNamesForThisDatatype(TxnId txnId, DataverseName dataverseName,
             String datatypeName) throws AlgebricksException {
         // Return all field types that aren't builtin types
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 0bf8c3d..74e8e61 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -34,11 +34,17 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
@@ -110,6 +116,16 @@ public class MetadataTransactionContext extends MetadataCache {
         logAndApply(new MetadataLogicalOperation(function, true));
     }
 
+    public void addFullTextFilter(FullTextFilterMetadataEntity filterMetadataEntity) {
+        droppedCache.dropFullTextFilter(filterMetadataEntity);
+        logAndApply(new MetadataLogicalOperation(filterMetadataEntity, true));
+    }
+
+    public void addFullTextConfig(FullTextConfigMetadataEntity configMetadataEntity) {
+        droppedCache.dropFullTextConfig(configMetadataEntity);
+        logAndApply(new MetadataLogicalOperation(configMetadataEntity, true));
+    }
+
     public void addAdapter(DatasourceAdapter adapter) {
         droppedCache.dropAdapterIfExists(adapter);
         logAndApply(new MetadataLogicalOperation(adapter, true));
@@ -164,6 +180,31 @@ public class MetadataTransactionContext extends MetadataCache {
         logAndApply(new MetadataLogicalOperation(function, false));
     }
 
+    public void dropFullTextConfig(DataverseName dataverseName, String configName) {
+        FullTextConfigDescriptor config = new FullTextConfigDescriptor(dataverseName, configName, null, null);
+        FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(config);
+
+        droppedCache.addFullTextConfigIfNotExists(configMetadataEntity);
+        logAndApply(new MetadataLogicalOperation(configMetadataEntity, false));
+    }
+
+    public void dropFullTextFilter(DataverseName dataverseName, String filterName) {
+        AbstractFullTextFilterDescriptor filter = new AbstractFullTextFilterDescriptor(dataverseName, filterName) {
+            @Override
+            public FullTextFilterType getFilterType() {
+                return null;
+            }
+
+            @Override
+            public IFullTextFilterEvaluatorFactory createEvaluatorFactory() {
+                return null;
+            }
+        };
+        FullTextFilterMetadataEntity filterMetadataEntity = new FullTextFilterMetadataEntity(filter);
+        droppedCache.addFullTextFilterIfNotExists(filterMetadataEntity);
+        logAndApply(new MetadataLogicalOperation(filterMetadataEntity, false));
+    }
+
     public void dropAdapter(DataverseName dataverseName, String adapterName) {
         AdapterIdentifier adapterIdentifier = new AdapterIdentifier(dataverseName, adapterName);
         DatasourceAdapter adapter = new DatasourceAdapter(adapterIdentifier, null, null, null, null);
@@ -218,6 +259,14 @@ public class MetadataTransactionContext extends MetadataCache {
         return droppedCache.getFunction(functionSignature) != null;
     }
 
+    public boolean fullTextConfigIsDropped(DataverseName dataverseName, String configName) {
+        return droppedCache.getFullTextConfig(dataverseName, configName) != null;
+    }
+
+    public boolean fullTextFilterIsDropped(DataverseName dataverseName, String filterName) {
+        return droppedCache.getFullTextFilter(dataverseName, filterName) != null;
+    }
+
     public List<MetadataLogicalOperation> getOpLog() {
         return opLog;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 1646a93..970a4e0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -36,6 +36,8 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
@@ -420,6 +422,81 @@ public interface IMetadataManager extends IMetadataBootstrap {
     /**
      * @param mdTxnCtx
      *            MetadataTransactionContext of an active metadata transaction.
+     * @param filterMetadataEntity
+     *            the full-text filter descriptor to be added
+     * @throws AlgebricksException
+     *              For example, if the filter with the same name in the same dataverse already exists
+     */
+    void addFullTextFilter(MetadataTransactionContext mdTxnCtx, FullTextFilterMetadataEntity filterMetadataEntity)
+            throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param filterName
+     *            the name of the full-text filter to be fetched
+     * @throws AlgebricksException
+     *              For example, if the filter doesn't exist
+     */
+    FullTextFilterMetadataEntity getFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+            String filterName) throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param filterName
+     *            the name of the full-text filter to be dropped
+     * @throws AlgebricksException
+     *              For example, if ifExists is set to false and the filter doesn't exist
+     */
+    void dropFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String filterName)
+            throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param configMetadataEntity
+     *            the full-text config descriptor to be added
+     * @throws AlgebricksException
+     *              For example, if the config with the same name in the same dataverse already exists
+     */
+    void addFullTextConfig(MetadataTransactionContext mdTxnCtx, FullTextConfigMetadataEntity configMetadataEntity)
+            throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param configName
+     *            the name of the full-text config to be fetched
+     * @throws AlgebricksException
+     *              For example, if the full-text config doesn't exist
+     * @return
+     */
+    FullTextConfigMetadataEntity getFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+            String configName) throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param configName
+     *            the name of the full-text config to be dropped
+     * @throws AlgebricksException
+     *              For example, if ifExists is set to false and the config doesn't exist
+     */
+    void dropFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String configName)
+            throws AlgebricksException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
      * @param adapter
      *            An instance of type Adapter that represents the adapter being
      *            added
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index bca4171..ef1f45d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -36,6 +36,8 @@ import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
@@ -447,6 +449,80 @@ public interface IMetadataNode extends Remote, Serializable {
 
     /**
      * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param filterMetadataEntity
+     *            the full-text filter metadata entity to be added
+     * @throws AlgebricksException
+     *              For example, if the filter with the same name in the same dataverse already exists
+     */
+    void addFullTextFilter(TxnId txnId, FullTextFilterMetadataEntity filterMetadataEntity)
+            throws RemoteException, AlgebricksException;
+
+    /**
+     * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param filterName
+     *            the name of the full-text filter to be fetched
+     * @throws AlgebricksException
+     *              For example, if the filter doesn't exist
+     */
+    FullTextFilterMetadataEntity getFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+            throws RemoteException, AlgebricksException;
+
+    /**
+     * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param filterName
+     *            the name of the full-text filter to be dropped
+     * @throws AlgebricksException
+     *              For example, if ifExists is set to false and the filter doesn't exist
+     */
+    void dropFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+            throws RemoteException, AlgebricksException;
+
+    /**
+     * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param configMetadataEntity
+     *            the full-text config descriptor to be added
+     * @throws AlgebricksException
+     *              For example, if the config with the same name in the same dataverse already exists
+     */
+    void addFullTextConfig(TxnId txnId, FullTextConfigMetadataEntity configMetadataEntity)
+            throws AlgebricksException, RemoteException;
+
+    /**
+     * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param configName
+     *            the name of the full-text config to be fetched
+     * @throws AlgebricksException
+     *              For example, if the full-text config doesn't exist
+     */
+    FullTextConfigMetadataEntity getFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+            throws AlgebricksException, RemoteException;
+
+    /**
+     * @param txnId
+     *            Metadata transaction id of an active metadata transaction.
+     * @param dataverseName
+     *            the name of the dataverse where the full-text filter belongs
+     * @param configName
+     *            the name of the full-text config to be dropped
+     * @throws AlgebricksException
+     *              For example, if ifExists is set to false and the config doesn't exist
+     */
+    void dropFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+            throws RemoteException, AlgebricksException;
+
+    /**
+     * @param txnId
      * @param dataverseName
      * @return List<Adapter> A list containing the adapters in the specified
      *         dataverse
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index cf089fa..84da2e2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -122,7 +122,8 @@ public class MetadataBootstrap {
                     MetadataPrimaryIndexes.FEED_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
                     MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
                     MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
-                    MetadataPrimaryIndexes.SYNONYM_DATASET };
+                    MetadataPrimaryIndexes.SYNONYM_DATASET, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET,
+                    MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET };
 
     private MetadataBootstrap() {
     }
@@ -172,6 +173,7 @@ public class MetadataBootstrap {
             } else {
                 insertNewCompactionPoliciesIfNotExist(mdTxnCtx);
                 insertSynonymEntitiesIfNotExist(mdTxnCtx);
+                insertFullTextConfigAndFilterIfNotExist(mdTxnCtx);
             }
             // #. initialize datasetIdFactory
             MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
@@ -313,6 +315,38 @@ public class MetadataBootstrap {
         }
     }
 
+    // For backward-compatibility: for old datasets created by an older version of AsterixDB, they
+    // 1) may not have such a full-text config dataset in the metadata catalog,
+    // 2) may not have the default full-text config as an entry in the metadata catalog
+    // So here, let's try to insert if not exists
+    private static void insertFullTextConfigAndFilterIfNotExist(MetadataTransactionContext mdTxnCtx)
+            throws AlgebricksException {
+
+        // We need to insert data types first because datasets depend on data types
+        // ToDo: create a new function to reduce duplicated code here: addDatatypeIfNotExist()
+        IAType fullTextConfigRecordType = MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET.getPayloadRecordType();
+        if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                fullTextConfigRecordType.getTypeName()) == null) {
+            MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+                    fullTextConfigRecordType.getTypeName(), fullTextConfigRecordType, false));
+        }
+        IAType fullTextFilterRecordType = MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET.getPayloadRecordType();
+        if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                fullTextFilterRecordType.getTypeName()) == null) {
+            MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+                    fullTextFilterRecordType.getTypeName(), fullTextFilterRecordType, false));
+        }
+
+        if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME) == null) {
+            insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET });
+        }
+        if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME) == null) {
+            insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET });
+        }
+    }
+
     private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws AlgebricksException {
         try {
             String adapterName =
@@ -510,7 +544,11 @@ public class MetadataBootstrap {
     }
 
     private static void ensureCatalogUpgradability(IMetadataIndex index) {
-        if (index != MetadataPrimaryIndexes.SYNONYM_DATASET) {
+        if (index != MetadataPrimaryIndexes.SYNONYM_DATASET
+                // Backward-compatibility: FULLTEXT_ENTITY_DATASET is added to AsterixDB recently
+                // and may not exist in an older dataverse
+                && index != MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET
+                && index != MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET) {
             throw new IllegalStateException(
                     "attempt to create metadata index " + index.getIndexName() + ". Index should already exist");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 556090a..a191e43 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -61,6 +61,10 @@ public class MetadataPrimaryIndexes {
             new MetadataIndexImmutableProperties(MetadataConstants.EXTERNAL_FILE_DATASET_NAME, 14, 14);
     public static final MetadataIndexImmutableProperties PROPERTIES_SYNONYM =
             new MetadataIndexImmutableProperties(MetadataConstants.SYNONYM_DATASET_NAME, 15, 15);
+    public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_CONFIG =
+            new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME, 16, 16);
+    public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_FILTER =
+            new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME, 17, 17);
 
     public static final IMetadataIndex DATAVERSE_DATASET =
             new MetadataIndex(PROPERTIES_DATAVERSE, 2, new IAType[] { BuiltinType.ASTRING },
@@ -142,6 +146,17 @@ public class MetadataPrimaryIndexes {
                             Arrays.asList(MetadataRecordTypes.FIELD_NAME_SYNONYM_NAME)),
                     0, MetadataRecordTypes.SYNONYM_RECORDTYPE, true, new int[] { 0, 1 });
 
+    public static final IMetadataIndex FULL_TEXT_CONFIG_DATASET =
+            new MetadataIndex(PROPERTIES_FULL_TEXT_CONFIG, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+                    Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+                            Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_CONFIG_NAME)),
+                    0, MetadataRecordTypes.FULL_TEXT_CONFIG_RECORDTYPE, true, new int[] { 0, 1 });
+    public static final IMetadataIndex FULL_TEXT_FILTER_DATASET =
+            new MetadataIndex(PROPERTIES_FULL_TEXT_FILTER, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+                    Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+                            Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_FILTER_NAME)),
+                    0, MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE, true, new int[] { 0, 1 });
+
     private MetadataPrimaryIndexes() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 8430f44..14798d1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -109,6 +109,13 @@ public final class MetadataRecordTypes {
     public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
     public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
     public static final String FIELD_NAME_WHERE_CLAUSE = "WhereClause";
+    public static final String FIELD_NAME_FULL_TEXT_CONFIG_NAME = "FullTextConfigName";
+    public static final String FIELD_NAME_FULL_TEXT_FILTER_NAME = "FullTextFilterName";
+    public static final String FIELD_NAME_FULL_TEXT_FILTER_TYPE = "FullTextFilterType";
+    public static final String FIELD_NAME_FULL_TEXT_TOKENIZER = "Tokenizer";
+    public static final String FIELD_NAME_FULL_TEXT_FILTER_PIPELINE = "FullTextFilterPipeline";
+    public static final String FIELD_NAME_FULL_TEXT_STOPWORD_LIST = "StopwordList";
+    public static final String FIELD_NAME_FULL_TEXT_STEMMER_LANGUAGE = "Language";
 
     //---------------------------------- Record Types Creation ----------------------------------//
     //--------------------------------------- Properties ----------------------------------------//
@@ -509,6 +516,41 @@ public final class MetadataRecordTypes {
             //IsOpen?
             true);
 
+    //---------------------------------- FullText Config and Filter -------------------------//
+    public static final int FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+    // FullText Config
+    public static final int FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX = 1;
+    public static final int FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX = 2;
+    public static final int FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX = 3;
+
+    // FullText Filter
+    public static final int FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX = 1;
+    public static final int FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX = 2;
+
+    // Stopword Filter
+    public static final int FULLTEXT_ENTITY_ARECORD_STOPWORD_LIST_FIELD_INDEX = 3;
+
+    // Stemmer Filter
+    public static final int FULLTEXT_ENTITY_ARECORD_STEMMER_LANGUAGE_FIELD_INDEX = 3;
+
+    public static final String RECORD_NAME_FULL_TEXT_CONFIG = "FullTextConfigRecordType";
+    public static final ARecordType FULL_TEXT_CONFIG_RECORDTYPE = createRecordType(RECORD_NAME_FULL_TEXT_CONFIG,
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FULL_TEXT_CONFIG_NAME, FIELD_NAME_FULL_TEXT_TOKENIZER,
+                    FIELD_NAME_FULL_TEXT_FILTER_PIPELINE },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, AUnionType.createNullableType(BuiltinType.ASTRING),
+                    AUnionType
+                            .createNullableType(new AOrderedListType(BuiltinType.ASTRING, "FullTextFilterPipeline")) },
+            true);
+
+    public static final String RECORD_NAME_FULL_TEXT_FILTER = "FullTextFilterRecordType";
+    // Different filters may have different fields, e.g.
+    // stopwords filter has the stopwords list, so this type is OPEN
+    public static final ARecordType FULL_TEXT_FILTER_RECORDTYPE = createRecordType(RECORD_NAME_FULL_TEXT_FILTER,
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FULL_TEXT_FILTER_NAME,
+                    FIELD_NAME_FULL_TEXT_FILTER_TYPE },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, true);
+
     // private members
     private MetadataRecordTypes() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 6ba8ba8..dd8e44c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -34,6 +34,8 @@ import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.entities.Synonym;
@@ -44,6 +46,8 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 
+import com.google.common.base.Strings;
+
 public class MetadataManagerUtil {
 
     private MetadataManagerUtil() {
@@ -148,6 +152,21 @@ public class MetadataManagerUtil {
         return MetadataManager.INSTANCE.getSynonym(mdTxnCtx, dataverseName, synonymName);
     }
 
+    public static FullTextConfigMetadataEntity findFullTextConfigDescriptor(MetadataTransactionContext mdTxnCtx,
+            DataverseName dataverseName, String ftConfigName) throws AlgebricksException {
+        // If the config name is null, then the default config will be returned
+        if (Strings.isNullOrEmpty(ftConfigName)) {
+            return FullTextConfigMetadataEntity.getDefaultFullTextConfigMetadataEntity();
+        }
+
+        return MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, ftConfigName);
+    }
+
+    public static FullTextFilterMetadataEntity findFullTextFilterDescriptor(MetadataTransactionContext mdTxnCtx,
+            DataverseName dataverseName, String ftFilterName) throws AlgebricksException {
+        return MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, ftFilterName);
+    }
+
     public static List<Index> getDatasetIndexes(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
             String datasetName) throws AlgebricksException {
         return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index e5fd6f7..3b387d8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -81,12 +81,15 @@ import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.FullTextUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -156,6 +159,7 @@ import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeBatchPointSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.common.IStorageManager;
@@ -445,6 +449,16 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
         return MetadataManagerUtil.findSynonym(mdTxnCtx, dataverseName, synonymName);
     }
 
+    public FullTextConfigMetadataEntity findFullTextConfig(DataverseName dataverseName, String ftConfigName)
+            throws AlgebricksException {
+        return MetadataManagerUtil.findFullTextConfigDescriptor(mdTxnCtx, dataverseName, ftConfigName);
+    }
+
+    public FullTextFilterMetadataEntity findFullTextFilter(DataverseName dataverseName, String ftFilterName)
+            throws AlgebricksException {
+        return MetadataManagerUtil.findFullTextFilterDescriptor(mdTxnCtx, dataverseName, ftFilterName);
+    }
+
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return BuiltinFunctions.getBuiltinFunctionInfo(fid);
@@ -1646,6 +1660,9 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
 
             IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
                     secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+                    FullTextUtil.fetchFilterAndCreateConfigEvaluator(this, secondaryIndex.getDataverseName(),
+                            secondaryIndex.getFullTextConfigName());
 
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                     getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
@@ -1687,8 +1704,9 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
                 keyFields[k] = k;
             }
 
-            tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
-                    keyFields, isPartitioned, true, false, MissingWriterFactory.INSTANCE);
+            tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory,
+                    fullTextConfigEvaluatorFactory, docField, keyFields, isPartitioned, true, false,
+                    MissingWriterFactory.INSTANCE);
             return new Pair<>(tokenizerOp, splitsAndConstraint.second);
         } catch (Exception e) {
             throw new AlgebricksException(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
new file mode 100644
index 0000000..3ee7106
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+
+public class FullTextConfigMetadataEntity implements IMetadataEntity<FullTextConfigMetadataEntity> {
+    private final FullTextConfigDescriptor fullTextConfig;
+
+    public FullTextConfigMetadataEntity(FullTextConfigDescriptor config) {
+        this.fullTextConfig = config;
+    }
+
+    public FullTextConfigDescriptor getFullTextConfig() {
+        return fullTextConfig;
+    }
+
+    @Override
+    public FullTextConfigMetadataEntity addToCache(MetadataCache cache) {
+        return cache.addFullTextConfigIfNotExists(this);
+    }
+
+    @Override
+    public FullTextConfigMetadataEntity dropFromCache(MetadataCache cache) {
+        return cache.dropFullTextConfig(this);
+    }
+
+    public static FullTextConfigMetadataEntity getDefaultFullTextConfigMetadataEntity() {
+        return new FullTextConfigMetadataEntity(FullTextConfigDescriptor.getDefaultFullTextConfig());
+    }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
new file mode 100644
index 0000000..eb4c7bc
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+
+public class FullTextFilterMetadataEntity implements IMetadataEntity<FullTextFilterMetadataEntity> {
+    private final AbstractFullTextFilterDescriptor filterDescriptor;
+
+    public FullTextFilterMetadataEntity(AbstractFullTextFilterDescriptor filterDescriptor) {
+        this.filterDescriptor = filterDescriptor;
+    }
+
+    @Override
+    public FullTextFilterMetadataEntity addToCache(MetadataCache cache) {
+        return cache.addFullTextFilterIfNotExists(this);
+    }
+
+    @Override
+    public FullTextFilterMetadataEntity dropFromCache(MetadataCache cache) {
+        return cache.dropFullTextFilter(this);
+    }
+
+    public AbstractFullTextFilterDescriptor getFullTextFilter() {
+        return filterDescriptor;
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index ee8622c..c8133e0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -41,7 +41,7 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
  */
 public class Index implements IMetadataEntity<Index>, Comparable<Index> {
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
     public static final int RECORD_INDICATOR = 0;
 
     private final DataverseName dataverseName;
@@ -58,12 +58,17 @@ public class Index implements IMetadataEntity<Index>, Comparable<Index> {
     private final boolean isPrimaryIndex;
     // Specific to NGRAM indexes.
     private final int gramLength;
+    // ToDo: to allow index to access the full-text config in another dataverse,
+    //   maybe we need to add a new field here fullTextConfigDataverseName for the dataverse name of the full-text config
+    // Specific to FullText indexes.
+    private final String fullTextConfigName;
     // Type of pending operations with respect to atomic DDL operation
     private int pendingOp;
 
     public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
             List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
-            int gramLength, boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
+            int gramLength, String fullTextConfigName, boolean overrideKeyFieldTypes, boolean isEnforced,
+            boolean isPrimaryIndex, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.indexName = indexName;
@@ -72,6 +77,8 @@ public class Index implements IMetadataEntity<Index>, Comparable<Index> {
         this.keyFieldSourceIndicators = keyFieldSourceIndicators;
         this.keyFieldTypes = keyFieldTypes;
         this.gramLength = gramLength;
+        this.fullTextConfigName = fullTextConfigName;
+
         this.overrideKeyFieldTypes = overrideKeyFieldTypes;
         this.isEnforced = isEnforced;
         this.isPrimaryIndex = isPrimaryIndex;
@@ -82,7 +89,7 @@ public class Index implements IMetadataEntity<Index>, Comparable<Index> {
             List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
             boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
         this(dataverseName, datasetName, indexName, indexType, keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
-                -1, overrideKeyFieldTypes, isEnforced, isPrimaryIndex, pendingOp);
+                -1, null, overrideKeyFieldTypes, isEnforced, isPrimaryIndex, pendingOp);
     }
 
     public DataverseName getDataverseName() {
@@ -113,6 +120,10 @@ public class Index implements IMetadataEntity<Index>, Comparable<Index> {
         return gramLength;
     }
 
+    public String getFullTextConfigName() {
+        return fullTextConfigName;
+    }
+
     public IndexType getIndexType() {
         return indexType;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java
new file mode 100644
index 0000000..ec5cdc9
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java
@@ -0,0 +1,163 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX;
+
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.commons.lang3.EnumUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextConfigMetadataEntityTupleTranslator extends AbstractTupleTranslator<FullTextConfigMetadataEntity> {
+
+    private static final int FULL_TEXT_CONFIG_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+    protected final ArrayTupleReference tuple;
+    protected final ISerializerDeserializer<AInt8> int8Serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+
+    protected FullTextConfigMetadataEntityTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, FULL_TEXT_CONFIG_PAYLOAD_TUPLE_FIELD_INDEX);
+        if (getTuple) {
+            tuple = new ArrayTupleReference();
+        } else {
+            tuple = null;
+        }
+    }
+
+    @Override
+    protected FullTextConfigMetadataEntity createMetadataEntityFromARecord(ARecord aRecord)
+            throws HyracksDataException, AlgebricksException {
+        DataverseName dataverseName = DataverseName.createFromCanonicalForm(
+                ((AString) aRecord.getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX))
+                        .getStringValue());
+
+        String name = ((AString) aRecord.getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX))
+                .getStringValue();
+
+        TokenizerCategory tokenizerCategory =
+                EnumUtils.getEnumIgnoreCase(TokenizerCategory.class,
+                        ((AString) aRecord
+                                .getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX))
+                                        .getStringValue());
+
+        ImmutableList.Builder<String> filterNamesBuilder = ImmutableList.builder();
+        IACursor filterNamesCursor = ((AOrderedList) (aRecord
+                .getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX))).getCursor();
+        while (filterNamesCursor.next()) {
+            filterNamesBuilder.add(((AString) filterNamesCursor.get()).getStringValue());
+        }
+
+        FullTextConfigDescriptor configDescriptor =
+                new FullTextConfigDescriptor(dataverseName, name, tokenizerCategory, filterNamesBuilder.build());
+        FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(configDescriptor);
+        return configMetadataEntity;
+    }
+
+    private void writeIndex(String dataverseName, String configName, ArrayTupleBuilder tupleBuilder)
+            throws HyracksDataException {
+        aString.setValue(dataverseName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        aString.setValue(configName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(FullTextConfigMetadataEntity configMetadataEntity)
+            throws HyracksDataException {
+        tupleBuilder.reset();
+
+        FullTextConfigDescriptor configDescriptor = configMetadataEntity.getFullTextConfig();
+
+        writeIndex(configDescriptor.getDataverseName().getCanonicalForm(), configDescriptor.getName(), tupleBuilder);
+
+        recordBuilder.reset(MetadataRecordTypes.FULL_TEXT_CONFIG_RECORDTYPE);
+
+        // write dataverse name
+        fieldValue.reset();
+        aString.setValue(configDescriptor.getDataverseName().getCanonicalForm());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        // write name
+        fieldValue.reset();
+        aString.setValue(configDescriptor.getName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX, fieldValue);
+
+        // write tokenizer category
+        fieldValue.reset();
+        aString.setValue(configDescriptor.getTokenizerCategory().name());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX, fieldValue);
+
+        // set filter pipeline
+        List<String> filterNames = configDescriptor.getFilterNames();
+
+        OrderedListBuilder listBuilder = new OrderedListBuilder();
+        listBuilder.reset(new AOrderedListType(BuiltinType.ASTRING, null));
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+        for (String s : filterNames) {
+            itemValue.reset();
+            aString.setValue(s);
+            stringSerde.serialize(aString, itemValue.getDataOutput());
+            listBuilder.addItem(itemValue);
+        }
+
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX, fieldValue);
+
+        recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        tupleBuilder.addFieldEndOffset();
+
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java
new file mode 100644
index 0000000..4845310
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java
@@ -0,0 +1,201 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FIELD_NAME_FULL_TEXT_STOPWORD_LIST;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX;
+
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextFilterMetadataEntityTupleTranslator extends AbstractTupleTranslator<FullTextFilterMetadataEntity> {
+
+    private static final int FULLTEXT_FILTER_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+    protected final ArrayTupleReference tuple;
+    protected final ISerializerDeserializer<AInt8> int8Serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+
+    protected FullTextFilterMetadataEntityTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, FULLTEXT_FILTER_PAYLOAD_TUPLE_FIELD_INDEX);
+        if (getTuple) {
+            tuple = new ArrayTupleReference();
+        } else {
+            tuple = null;
+        }
+    }
+
+    @Override
+    protected FullTextFilterMetadataEntity createMetadataEntityFromARecord(ARecord aRecord) throws AlgebricksException {
+        AString dataverseName = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX);
+        AString filterName = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX);
+        AString filterTypeAString = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX);
+
+        FullTextFilterType filterType = FullTextFilterType.getEnumIgnoreCase(filterTypeAString.getStringValue());
+        AbstractFullTextFilterDescriptor filterDescriptor;
+        switch (filterType) {
+            case STOPWORDS:
+                return createStopwordsFilterDescriptorFromARecord(dataverseName, filterName, aRecord);
+            case STEMMER:
+            case SYNONYM:
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, "Not supported yet");
+        }
+    }
+
+    public FullTextFilterMetadataEntity createStopwordsFilterDescriptorFromARecord(AString dataverseName, AString name,
+            ARecord aRecord) {
+        ImmutableList.Builder<String> stopwordsBuilder = ImmutableList.<String> builder();
+        IACursor stopwordsCursor = ((AOrderedList) (aRecord
+                .getValueByPos(MetadataRecordTypes.FULLTEXT_ENTITY_ARECORD_STOPWORD_LIST_FIELD_INDEX))).getCursor();
+        while (stopwordsCursor.next()) {
+            stopwordsBuilder.add(((AString) stopwordsCursor.get()).getStringValue());
+        }
+
+        StopwordsFullTextFilterDescriptor filterDescriptor = new StopwordsFullTextFilterDescriptor(
+                DataverseName.createFromCanonicalForm(dataverseName.getStringValue()), name.getStringValue(),
+                stopwordsBuilder.build());
+        return new FullTextFilterMetadataEntity(filterDescriptor);
+    }
+
+    private void writeKeyAndValue2FieldVariables(String key, String value) throws HyracksDataException {
+        fieldName.reset();
+        aString.setValue(key);
+        stringSerde.serialize(aString, fieldName.getDataOutput());
+
+        fieldValue.reset();
+        aString.setValue(value);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+    }
+
+    private void writeOrderedList2RecordBuilder(String strFieldName, List<String> list) throws HyracksDataException {
+        fieldName.reset();
+        aString.setValue(strFieldName);
+        stringSerde.serialize(aString, fieldName.getDataOutput());
+
+        OrderedListBuilder listBuilder = new OrderedListBuilder();
+        listBuilder.reset(new AOrderedListType(BuiltinType.ASTRING, null));
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+        for (String s : list) {
+            itemValue.reset();
+            aString.setValue(s);
+            stringSerde.serialize(aString, itemValue.getDataOutput());
+            listBuilder.addItem(itemValue);
+        }
+
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+
+        recordBuilder.addField(fieldName, fieldValue);
+    }
+
+    private void writeStopwordFilterDescriptor(StopwordsFullTextFilterDescriptor stopwordsFullTextFilterDescriptor)
+            throws HyracksDataException {
+        writeOrderedList2RecordBuilder(FIELD_NAME_FULL_TEXT_STOPWORD_LIST,
+                stopwordsFullTextFilterDescriptor.getStopwordList());
+    }
+
+    private void writeFulltextFilter(AbstractFullTextFilterDescriptor filterDescriptor)
+            throws AsterixException, HyracksDataException {
+        fieldValue.reset();
+        aString.setValue(filterDescriptor.getDataverseName().getCanonicalForm());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+        fieldValue.reset();
+        aString.setValue(filterDescriptor.getName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX, fieldValue);
+
+        fieldValue.reset();
+        aString.setValue(filterDescriptor.getFilterType().getValue());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX, fieldValue);
+
+        switch (filterDescriptor.getFilterType()) {
+            case STOPWORDS:
+                writeStopwordFilterDescriptor((StopwordsFullTextFilterDescriptor) filterDescriptor);
+                break;
+            case STEMMER:
+            case SYNONYM:
+            default:
+                throw new AsterixException(ErrorCode.METADATA_ERROR, "Not supported yet");
+        }
+    }
+
+    private void writeIndex(String dataverseName, String filterName, ArrayTupleBuilder tupleBuilder)
+            throws HyracksDataException {
+        aString.setValue(dataverseName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        aString.setValue(filterName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(FullTextFilterMetadataEntity filterMetadataEntity)
+            throws HyracksDataException, AsterixException {
+        tupleBuilder.reset();
+
+        writeIndex(filterMetadataEntity.getFullTextFilter().getDataverseName().getCanonicalForm(),
+                filterMetadataEntity.getFullTextFilter().getName(), tupleBuilder);
+
+        // Write the record
+        recordBuilder.reset(MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE);
+
+        writeFulltextFilter(filterMetadataEntity.getFullTextFilter());
+
+        recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        tupleBuilder.addFieldEndOffset();
+
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 79ce2a4..c69c0b1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -54,6 +54,8 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
+import com.google.common.base.Strings;
+
 /**
  * Translates an Index metadata entity to an ITupleReference and vice versa.
  */
@@ -64,6 +66,7 @@ public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
 
     // Field name of open field.
     public static final String GRAM_LENGTH_FIELD_NAME = "GramLength";
+    public static final String FULL_TEXT_CONFIG_FIELD_NAME = "FullTextConfig";
     public static final String INDEX_SEARCHKEY_TYPE_FIELD_NAME = "SearchKeyType";
     public static final String INDEX_ISENFORCED_FIELD_NAME = "IsEnforced";
     public static final String INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME = "SearchKeySourceIndicator";
@@ -156,6 +159,12 @@ public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
             gramLength = ((AInt32) indexRecord.getValueByPos(gramLenPos)).getIntegerValue();
         }
 
+        String fullTextConfig = null;
+        int fullTextConfigPos = indexRecord.getType().getFieldIndex(FULL_TEXT_CONFIG_FIELD_NAME);
+        if (fullTextConfigPos >= 0) {
+            fullTextConfig = ((AString) indexRecord.getValueByPos(fullTextConfigPos)).getStringValue();
+        }
+
         // Read a field-source-indicator field.
         List<Integer> keyFieldSourceIndicator = new ArrayList<>();
         int keyFieldSourceIndicatorIndex =
@@ -190,7 +199,8 @@ public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
         }
 
         return new Index(dataverseName, datasetName, indexName, indexStructure, searchKey, keyFieldSourceIndicator,
-                searchKeyType, gramLength, isOverridingKeyTypes, isEnforcingKeys, isPrimaryIndex, pendingOp);
+                searchKeyType, gramLength, fullTextConfig, isOverridingKeyTypes, isEnforcingKeys, isPrimaryIndex,
+                pendingOp);
     }
 
     @Override
@@ -292,6 +302,7 @@ public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
      */
     protected void writeOpenFields(Index index) throws HyracksDataException {
         writeGramLength(index);
+        writeFullTextConfig(index);
         writeSearchKeyType(index);
         writeEnforced(index);
         writeSearchKeySourceIndicator(index);
@@ -308,6 +319,20 @@ public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
         }
     }
 
+    private void writeFullTextConfig(Index index) throws HyracksDataException {
+        if (!Strings.isNullOrEmpty(index.getFullTextConfigName())) {
+            nameValue.reset();
+            aString.setValue(FULL_TEXT_CONFIG_FIELD_NAME);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+
+            fieldValue.reset();
+            aString.setValue(index.getFullTextConfigName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+
+            recordBuilder.addField(nameValue, fieldValue);
+        }
+    }
+
     private void writeSearchKeyType(Index index) throws HyracksDataException {
         if (index.isOverridingKeyFieldTypes()) {
             OrderedListBuilder typeListBuilder = new OrderedListBuilder();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
index 1079904..c2a9ee3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -65,6 +65,14 @@ public class MetadataTupleTranslatorProvider {
         return new FunctionTupleTranslator(txnId, metadataNode, getTuple);
     }
 
+    public FullTextConfigMetadataEntityTupleTranslator getFullTextConfigTupleTranslator(boolean getTuple) {
+        return new FullTextConfigMetadataEntityTupleTranslator(getTuple);
+    }
+
+    public FullTextFilterMetadataEntityTupleTranslator getFullTextFilterTupleTranslator(boolean getTuple) {
+        return new FullTextFilterMetadataEntityTupleTranslator(getTuple);
+    }
+
     public IndexTupleTranslator getIndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
         return new IndexTupleTranslator(txnId, metadataNode, getTuple);
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
index 2fb3ae6..2ca646f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
@@ -34,6 +34,8 @@ final class MetadataLockKey implements IMetadataLock.LockKey {
         EXTENSION,
         FEED_POLICY,
         FUNCTION,
+        FULL_TEXT_CONFIG,
+        FULL_TEXT_FILTER,
         LIBRARY,
         ADAPTER,
         MERGE_POLICY,
@@ -108,6 +110,14 @@ final class MetadataLockKey implements IMetadataLock.LockKey {
         return new MetadataLockKey(EntityKind.FUNCTION, null, dataverseName, functionName);
     }
 
+    static MetadataLockKey createFullTextConfigLockKey(DataverseName dataverseName, String fullTextConfigName) {
+        return new MetadataLockKey(EntityKind.FULL_TEXT_CONFIG, null, dataverseName, fullTextConfigName);
+    }
+
+    static MetadataLockKey createFullTextFilterLockKey(DataverseName dataverseName, String fullTextFilterName) {
+        return new MetadataLockKey(EntityKind.FULL_TEXT_FILTER, null, dataverseName, fullTextFilterName);
+    }
+
     static MetadataLockKey createLibraryLockKey(DataverseName dataverseName, String libraryName) {
         return new MetadataLockKey(EntityKind.LIBRARY, null, dataverseName, libraryName);
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
index 63d01d1..48508e1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
@@ -110,6 +110,38 @@ public class MetadataLockManager implements IMetadataLockManager {
     }
 
     @Override
+    public void acquireFullTextConfigReadLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createFullTextConfigLockKey(dataverseName, fullTextConfigName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.READ, lock);
+    }
+
+    @Override
+    public void acquireFullTextConfigWriteLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createFullTextConfigLockKey(dataverseName, fullTextConfigName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.WRITE, lock);
+    }
+
+    @Override
+    public void acquireFullTextFilterReadLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createFullTextFilterLockKey(dataverseName, fullTextFilterName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.READ, lock);
+    }
+
+    @Override
+    public void acquireFullTextFilterWriteLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createFullTextFilterLockKey(dataverseName, fullTextFilterName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.WRITE, lock);
+    }
+
+    @Override
     public void acquireLibraryReadLock(LockList locks, DataverseName dataverseName, String libraryName)
             throws AlgebricksException {
         MetadataLockKey key = MetadataLockKey.createLibraryLockKey(dataverseName, libraryName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java
new file mode 100644
index 0000000..563bab4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextUtil {
+    public static IFullTextConfigEvaluatorFactory fetchFilterAndCreateConfigEvaluator(MetadataProvider metadataProvider,
+            DataverseName dataverseName, String configName) throws AlgebricksException {
+        FullTextConfigDescriptor configDescriptor =
+                metadataProvider.findFullTextConfig(dataverseName, configName).getFullTextConfig();
+
+        ImmutableList.Builder<AbstractFullTextFilterDescriptor> filterDescriptorsBuilder = ImmutableList.builder();
+        for (String filterName : configDescriptor.getFilterNames()) {
+            filterDescriptorsBuilder
+                    .add(metadataProvider.findFullTextFilter(dataverseName, filterName).getFullTextFilter());
+        }
+
+        return configDescriptor.createEvaluatorFactory(filterDescriptorsBuilder.build());
+    }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index f2b7558..43798b2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -48,6 +48,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.IResourceFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
@@ -119,12 +120,17 @@ public class InvertedIndexResourceFactoryProvider implements IResourceFactoryPro
         IBinaryComparatorFactory[] tokenCmpFactories =
                 getTokenComparatorFactories(dataset, index, recordType, metaType);
         IBinaryTokenizerFactory tokenizerFactory = getTokenizerFactory(dataset, index, recordType, metaType);
+        IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+                FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
+                        index.getFullTextConfigName());
+
         return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                isPartitioned, invertedIndexFields, secondaryFilterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+                fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
+                secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+                bloomFilterFalsePositiveRate);
     }
 
     // Returns an array of the type traits of the inverted list elements
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index 4489050..2875297 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -46,6 +46,8 @@ public class MetadataConstants {
     public static final String COMPACTION_POLICY_DATASET_NAME = "CompactionPolicy";
     public static final String EXTERNAL_FILE_DATASET_NAME = "ExternalFile";
     public static final String SYNONYM_DATASET_NAME = "Synonym";
+    public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
+    public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
 
     private MetadataConstants() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 4309c2e..7bb11e5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.metadata.utils;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
 import org.apache.asterix.common.metadata.DataverseName;
@@ -25,6 +29,9 @@ import org.apache.asterix.common.metadata.IMetadataLockUtil;
 import org.apache.asterix.common.metadata.LockList;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+
 public class MetadataLockUtil implements IMetadataLockUtil {
 
     @Override
@@ -89,9 +96,12 @@ public class MetadataLockUtil implements IMetadataLockUtil {
 
     @Override
     public void createIndexBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
-            String datasetName) throws AlgebricksException {
+            String datasetName, String fullTextConfigName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);
         lockMgr.acquireDatasetCreateIndexLock(locks, dataverseName, datasetName);
+        if (!Strings.isNullOrEmpty(fullTextConfigName)) {
+            lockMgr.acquireFullTextConfigReadLock(locks, dataverseName, fullTextConfigName);
+        }
     }
 
     @Override
@@ -157,6 +167,43 @@ public class MetadataLockUtil implements IMetadataLockUtil {
     }
 
     @Override
+    public void createFullTextFilterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String fullTextFilterName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFullTextFilterWriteLock(locks, dataverseName, fullTextFilterName);
+    }
+
+    @Override
+    public void dropFullTextFilterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String fullTextFilterName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFullTextFilterWriteLock(locks, dataverseName, fullTextFilterName);
+    }
+
+    @Override
+    public void createFullTextConfigBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String fullTextConfigName, ImmutableList<String> fullTextFilterNames) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFullTextConfigWriteLock(locks, dataverseName, fullTextConfigName);
+
+        // We should avoid sorting the original list, and the original list is immutable and cannot be sorted anyway
+        List<String> fullTextFilterNamesMutable = new ArrayList<>(fullTextFilterNames);
+
+        // sort the filters to guarantee locks are always fetched in the same order to avoid dead lock between filters
+        Collections.sort(fullTextFilterNamesMutable);
+        for (String filterName : fullTextFilterNamesMutable) {
+            lockMgr.acquireFullTextFilterReadLock(locks, dataverseName, filterName);
+        }
+    }
+
+    @Override
+    public void dropFullTextConfigBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String configName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireFullTextConfigWriteLock(locks, dataverseName, configName);
+    }
+
+    @Override
     public void createAdapterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
             String adapterName, DataverseName libraryDataverseName, String libraryName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index cf84e56..0b6821c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -55,6 +55,7 @@ import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescri
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class SecondaryCorrelatedInvertedIndexOperationsHelper extends SecondaryCorrelatedTreeIndexOperationsHelper {
@@ -64,6 +65,7 @@ public class SecondaryCorrelatedInvertedIndexOperationsHelper extends SecondaryC
     private IBinaryComparatorFactory[] tokenComparatorFactories;
     private ITypeTraits[] tokenTypeTraits;
     private IBinaryTokenizerFactory tokenizerFactory;
+    private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     // For tokenization, sorting and loading. Represents <token, primary keys>.
     private int numTokenKeyPairFields;
     private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
@@ -150,6 +152,8 @@ public class SecondaryCorrelatedInvertedIndexOperationsHelper extends SecondaryC
         // and add the choice to the index metadata.
         tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
                 index.getGramLength());
+        fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
+                index.getDataverseName(), index.getFullTextConfigName());
         // Type traits for inverted-list elements. Inverted lists contain
         // primary keys.
         invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -279,8 +283,8 @@ public class SecondaryCorrelatedInvertedIndexOperationsHelper extends SecondaryC
             keyFields[i] = i + numSecondaryKeys;
         }
         BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
-                getTaggedRecordDescriptor(tokenKeyPairRecDesc), tokenizerFactory, docField, keyFields, isPartitioned,
-                false, true, MissingWriterFactory.INSTANCE);
+                getTaggedRecordDescriptor(tokenKeyPairRecDesc), tokenizerFactory, fullTextConfigEvaluatorFactory,
+                docField, keyFields, isPartitioned, false, true, MissingWriterFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
         return tokenizerOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 85205b3..6329d0d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -58,6 +58,7 @@ import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class SecondaryInvertedIndexOperationsHelper extends SecondaryTreeIndexOperationsHelper {
@@ -67,6 +68,7 @@ public class SecondaryInvertedIndexOperationsHelper extends SecondaryTreeIndexOp
     private IBinaryComparatorFactory[] tokenComparatorFactories;
     private ITypeTraits[] tokenTypeTraits;
     private IBinaryTokenizerFactory tokenizerFactory;
+    private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     // For tokenization, sorting and loading. Represents <token, primary keys>.
     private int numTokenKeyPairFields;
     private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
@@ -79,6 +81,8 @@ public class SecondaryInvertedIndexOperationsHelper extends SecondaryTreeIndexOp
     protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
             SourceLocation sourceLoc) throws AlgebricksException {
         super(dataset, index, metadataProvider, sourceLoc);
+        this.fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
+                index.getDataverseName(), index.getFullTextConfigName());
     }
 
     @Override
@@ -275,9 +279,9 @@ public class SecondaryInvertedIndexOperationsHelper extends SecondaryTreeIndexOp
         for (int i = 0; i < primaryKeyFields.length; i++) {
             primaryKeyFields[i] = numSecondaryKeys + i;
         }
-        BinaryTokenizerOperatorDescriptor tokenizerOp =
-                new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
-                        primaryKeyFields, isPartitioned, false, false, MissingWriterFactory.INSTANCE);
+        BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc,
+                tokenizerFactory, fullTextConfigEvaluatorFactory, docField, primaryKeyFields, isPartitioned, false,
+                false, MissingWriterFactory.INSTANCE);
         tokenizerOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 5fcb6e0..13524a2 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -73,7 +73,7 @@ public class IndexTupleTranslatorTest {
             Index index = new Index(dvTest, "d1", "i1", IndexType.BTREE,
                     Collections.singletonList(Collections.singletonList("row_id")),
                     indicator == null ? null : Collections.singletonList(indicator),
-                    Collections.singletonList(BuiltinType.AINT64), -1, false, false, false, 0);
+                    Collections.singletonList(BuiltinType.AINT64), -1, null, false, false, false, 0);
 
             MetadataNode mockMetadataNode = mock(MetadataNode.class);
             when(mockMetadataNode.getDatatype(any(), any(DataverseName.class), anyString())).thenReturn(new Datatype(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index c644bbe..2dfc603 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -23,6 +23,7 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
@@ -96,4 +97,15 @@ public class AOrderedListBinaryTokenizer implements IBinaryTokenizer {
     public TokenizerType getTokenizerType() {
         return TokenizerType.LIST;
     }
+
+    @Override
+    public TokenizerCategory getTokenizerCategory() {
+        // Currently, the AOrderedListBinaryTokenizer is used in ftcontains() function only,
+        // which means it is used to tokenize WORD only
+        //
+        // The functions that utilize NGRAM index are edit_distance(), edit_distance_check() and so on,
+        // which don't take a list as input (string only).
+        // e.g. edit_distance(u.name, "Suzanna Tilson")
+        return TokenizerCategory.WORD;
+    }
 }
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 6b8bf71..6672b38 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -85,10 +85,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-std</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java
similarity index 82%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java
index ff0a9f6..3ffce9b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java
@@ -29,14 +29,16 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -47,11 +49,14 @@ import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.BinaryEntry;
 import org.apache.hyracks.data.std.util.BinaryHashSet;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
-public class FullTextContainsEvaluator implements IScalarEvaluator {
+public class FullTextContainsFunctionEvaluator implements IScalarEvaluator {
 
     // assuming type indicator in serde format
     protected static final int TYPE_INDICATOR_SIZE = 1;
@@ -68,6 +73,8 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
     protected IPointable outRight = VoidPointable.FACTORY.createPointable();
     protected IPointable[] outOptions;
     protected int optionArgsLength;
+    // By default, we conduct a conjunctive search.
+    protected FullTextContainsFunctionDescriptor.SearchMode mode = FullTextContainsFunctionDescriptor.SearchMode.ALL;
 
     // To conduct a full-text search, we convert all strings to the lower case.
     // In addition, since each token does not include the length information (2 bytes) in the beginning,
@@ -77,8 +84,9 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_TOKEN_POINTABLE_INSTANCE.createBinaryComparator();
     private final IBinaryComparator strLowerCaseCmp =
             BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE.createBinaryComparator();
-    private IBinaryTokenizer tokenizerForLeftArray = null;
-    private IBinaryTokenizer tokenizerForRightArray = null;
+
+    private IFullTextConfigEvaluator ftEvaluatorLeft;
+    private IFullTextConfigEvaluator ftEvaluatorRight;
 
     // Case insensitive hash for full-text search
     private IBinaryHashFunction hashFunc = null;
@@ -109,15 +117,23 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
     protected ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public FullTextContainsEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext context)
-            throws HyracksDataException {
+    public FullTextContainsFunctionEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) throws HyracksDataException {
+
         evalLeft = args[0].createScalarEvaluator(context);
         evalRight = args[1].createScalarEvaluator(context);
         optionArgsLength = args.length - 2;
         this.evalOptions = new IScalarEvaluator[optionArgsLength];
         this.outOptions = new IPointable[optionArgsLength];
         this.argOptions = new TaggedValuePointable[optionArgsLength];
-        // Full-text search options
+
+        // We need to have two dedicated ftEvaluatorLeft and ftEvaluatorRight to let them have dedicated tokenizers.
+        //
+        // ftEvaluatorLeft and ftEvaluatorRight are shared by multiple threads on the NC node,
+        // so each thread needs a local copy of them here
+        this.ftEvaluatorLeft = fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+        this.ftEvaluatorRight = fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
         for (int i = 0; i < optionArgsLength; i++) {
             this.evalOptions[i] = args[i + 2].createScalarEvaluator(context);
             this.outOptions[i] = VoidPointable.FACTORY.createPointable();
@@ -182,9 +198,10 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
         try {
             ABoolean b = fullTextContainsWithArg(typeTag2, argLeft, argRight) ? ABoolean.TRUE : ABoolean.FALSE;
             serde.serialize(b, out);
-        } catch (HyracksDataException e1) {
-            throw HyracksDataException.create(e1);
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e, ErrorCode.ERROR_PROCESSING_TUPLE);
         }
+
         result.set(resultStorage);
     }
 
@@ -197,10 +214,14 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
      * After traversing all tokens and still the foundCount is less than the given threshold, then returns false.
      */
     private boolean fullTextContainsWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2)
-            throws HyracksDataException {
+            throws HyracksDataException, AlgebricksException {
+        // The main logic
+
         // Since a fulltext search form is "ftcontains(X,Y,options)",
         // X (document) is the left side and Y (query predicate) is the right side.
 
+        setFullTextOption(argOptions);
+
         // Initialize variables that are required to conduct full-text search. (e.g., hash-set, tokenizer ...)
         if (rightHashSet == null) {
             initializeFullTextContains();
@@ -234,11 +255,14 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
         // Parameter: number of bucket, frame size, hashFunction, Comparator, byte array
         // that contains the key (this array will be set later.)
         rightHashSet = new BinaryHashSet(HASH_SET_SLOT_SIZE, HASH_SET_FRAME_SIZE, hashFunc, strLowerCaseTokenCmp, null);
-        tokenizerForLeftArray = BinaryTokenizerFactoryProvider.INSTANCE
-                .getWordTokenizerFactory(ATypeTag.STRING, false, true).createTokenizer();
+        IBinaryTokenizer tokenizerForLeftArray = BinaryTokenizerFactoryProvider.INSTANCE
+                .getWordTokenizerFactory(ATypeTag.STRING, true, true).createTokenizer();
+        ftEvaluatorLeft.setTokenizer(tokenizerForLeftArray);
     }
 
     void resetQueryArrayAndRight(byte[] arg2Array, ATypeTag typeTag2, IPointable arg2) throws HyracksDataException {
+
+        IBinaryTokenizer tokenizerForRightArray = null;
         // If the right side is an (un)ordered list, we need to apply the (un)ordered list tokenizer.
         switch (typeTag2) {
             case ARRAY:
@@ -256,6 +280,7 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             default:
                 break;
         }
+        ftEvaluatorRight.setTokenizer(tokenizerForRightArray);
 
         queryArray = arg2Array;
         queryArrayStartOffset = arg2.getStartOffset();
@@ -279,17 +304,18 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             queryArrayStartOffset = queryArrayStartOffset + numBytesToStoreLength;
             queryArrayLength = queryArrayLength - numBytesToStoreLength;
         }
-        tokenizerForRightArray.reset(queryArray, queryArrayStartOffset, queryArrayLength);
+        ftEvaluatorRight.reset(queryArray, queryArrayStartOffset, queryArrayLength);
 
         // Create tokens from the given query predicate
-        while (tokenizerForRightArray.hasNext()) {
-            tokenizerForRightArray.next();
+        while (ftEvaluatorRight.hasNext()) {
+            ftEvaluatorRight.next();
             queryTokenCount++;
 
+            IToken token = ftEvaluatorRight.getToken();
             // Insert the starting position and the length of the current token into the hash set.
             // We don't store the actual value of this token since we can access it via offset and length.
-            int tokenOffset = tokenizerForRightArray.getToken().getStartOffset();
-            int tokenLength = tokenizerForRightArray.getToken().getTokenLength();
+            int tokenOffset = token.getStartOffset();
+            int tokenLength = token.getTokenLength();
 
             // If a token comes from a string tokenizer, each token doesn't have the length data
             // in the beginning. Instead, if a token comes from an (un)ordered list, each token has
@@ -298,9 +324,8 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             // e.g., 8database <--- we only need to store the offset of 'd' and length 8.
             if (typeTag2 == ATypeTag.ARRAY || typeTag2 == ATypeTag.MULTISET) {
                 // How many bytes are required to store the length of the given token?
-                numBytesToStoreLength = UTF8StringUtil.getNumBytesToStoreLength(
-                        UTF8StringUtil.getUTFLength(tokenizerForRightArray.getToken().getData(),
-                                tokenizerForRightArray.getToken().getStartOffset()));
+                numBytesToStoreLength = UTF8StringUtil
+                        .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(token.getData(), token.getStartOffset()));
                 tokenOffset = tokenOffset + numBytesToStoreLength;
                 tokenLength = tokenLength - numBytesToStoreLength;
             }
@@ -310,7 +335,7 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             // Currently, for the full-text search, we don't support a phrase search yet.
             // So, each query predicate should have only one token.
             // The same logic should be applied in AbstractTOccurrenceSearcher() class.
-            checkWhetherFullTextPredicateIsPhrase(typeTag2, queryArray, tokenOffset, tokenLength, queryTokenCount);
+            checkWhetherFullTextPredicateIsPhrase(typeTag2, token.getData(), tokenOffset, tokenLength, queryTokenCount);
 
             // Count the number of tokens in the given query. We only count the unique tokens.
             // We only care about the first insertion of the token into the hash set
@@ -320,15 +345,18 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
             // Thus, when we find the current token (we don't increase the count in this case),
             // it should not exist.
             if (rightHashSet.find(keyEntry, queryArray, false) == -1) {
+                rightHashSet.setRefArray(token.getData());
                 rightHashSet.put(keyEntry);
                 uniqueQueryTokenCount++;
             }
-
         }
 
-        // Apply the full-text search option here
         // Based on the search mode option - "any" or "all", set the occurrence threshold of tokens.
-        setFullTextOption(argOptions, uniqueQueryTokenCount);
+        if (mode == FullTextContainsFunctionDescriptor.SearchMode.ANY) {
+            occurrenceThreshold = 1;
+        } else {
+            occurrenceThreshold = uniqueQueryTokenCount;
+        }
     }
 
     private void checkWhetherFullTextPredicateIsPhrase(ATypeTag typeTag, byte[] refArray, int tokenOffset,
@@ -359,21 +387,22 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
      * Sets the full-text options. The odd element is an option name and the even element is the argument
      * for that option. (e.g., argOptions[0] = "mode", argOptions[1] = "all")
      */
-    private void setFullTextOption(IPointable[] argOptions, int uniqueQueryTokenCount) throws HyracksDataException {
-        // By default, we conduct a conjunctive search.
-        occurrenceThreshold = uniqueQueryTokenCount;
+    private void setFullTextOption(IPointable[] argOptions) throws HyracksDataException {
+        // Maybe using a JSON parser here can make things easier?
         for (int i = 0; i < optionArgsLength; i = i + 2) {
             // mode option
-            if (compareStrInByteArrayAndPointable(FullTextContainsDescriptor.getSearchModeOptionArray(), argOptions[i],
-                    true) == 0) {
-                if (compareStrInByteArrayAndPointable(FullTextContainsDescriptor.getDisjunctiveFTSearchOptionArray(),
-                        argOptions[i + 1], true) == 0) {
+            if (compareStrInByteArrayAndPointable(FullTextContainsFunctionDescriptor.getSearchModeOptionArray(),
+                    argOptions[i], true) == 0) {
+                if (compareStrInByteArrayAndPointable(
+                        FullTextContainsFunctionDescriptor.getDisjunctiveFTSearchOptionArray(), argOptions[i + 1],
+                        true) == 0) {
                     // ANY
-                    occurrenceThreshold = 1;
+                    mode = FullTextContainsFunctionDescriptor.SearchMode.ANY;
                 } else if (compareStrInByteArrayAndPointable(
-                        FullTextContainsDescriptor.getConjunctiveFTSearchOptionArray(), argOptions[i + 1], true) == 0) {
+                        FullTextContainsFunctionDescriptor.getConjunctiveFTSearchOptionArray(), argOptions[i + 1],
+                        true) == 0) {
                     // ALL
-                    occurrenceThreshold = uniqueQueryTokenCount;
+                    mode = FullTextContainsFunctionDescriptor.SearchMode.ALL;
                 }
             }
         }
@@ -388,26 +417,20 @@ public class FullTextContainsEvaluator implements IScalarEvaluator {
         // The left side: field (document)
         // Resets the tokenizer for the given keywords in a document.
 
-        // How many bytes are required to store the length of the given string?
-        int numBytesToStoreLength = UTF8StringUtil
-                .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(arg1.getByteArray(), arg1.getStartOffset()));
-        int startOffset = arg1.getStartOffset() + numBytesToStoreLength;
-        int length = arg1.getLength() - numBytesToStoreLength;
-
-        tokenizerForLeftArray.reset(arg1.getByteArray(), startOffset, length);
+        ftEvaluatorLeft.reset(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength());
 
         // Creates tokens from a field in the left side (document)
-        while (tokenizerForLeftArray.hasNext()) {
-            tokenizerForLeftArray.next();
+        while (ftEvaluatorLeft.hasNext()) {
+            ftEvaluatorLeft.next();
 
+            IToken token = ftEvaluatorLeft.getToken();
             // Records the starting position and the length of the current token.
-            keyEntry.set(tokenizerForLeftArray.getToken().getStartOffset(),
-                    tokenizerForLeftArray.getToken().getTokenLength());
+            keyEntry.set(token.getStartOffset(), token.getTokenLength());
 
             // Checks whether this token exists in the query hash-set.
             // We don't count multiple occurrence of a token now.
             // So, finding the same query predicate twice will not be counted as a found.
-            if (rightHashSet.find(keyEntry, arg1.getByteArray(), true) == 1) {
+            if (rightHashSet.find(keyEntry, token.getData(), true) == 1) {
                 foundCount++;
                 if (foundCount >= occurrenceThreshold) {
                     return true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java
similarity index 65%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java
index a3c3e5e..c6c2eed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java
@@ -26,45 +26,85 @@ import org.apache.asterix.common.annotations.MissingNullInOutFunction;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.FullTextContainsEvaluator;
+import org.apache.asterix.runtime.evaluators.common.FullTextContainsFunctionEvaluator;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
+// Descriptor for the ftcontains() function.
+//
+// This is different from the full-text config descriptor (FullTextConfigDescriptor)
+// which contains a tokenizer and full-text filter descriptors (AbstractFullTextFilterDescriptor).
 @MissingNullInOutFunction
-public class FullTextContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
+public class FullTextContainsFunctionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 2L;
 
     // parameter name and its type - based on the order of parameters in this map, parameters will be re-arranged.
     private static final Map<String, ATypeTag> paramTypeMap = new LinkedHashMap<>();
 
     public static final String SEARCH_MODE_OPTION = "mode";
-    public static final String DISJUNCTIVE_SEARCH_MODE_OPTION = "any";
-    public static final String CONJUNCTIVE_SEARCH_MODE_OPTION = "all";
-
     private static final byte[] SEARCH_MODE_OPTION_ARRAY = UTF8StringUtil.writeStringToBytes(SEARCH_MODE_OPTION);
     private static final byte[] DISJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
-            UTF8StringUtil.writeStringToBytes(DISJUNCTIVE_SEARCH_MODE_OPTION);
+            UTF8StringUtil.writeStringToBytes(SearchMode.ANY.getValue());
     private static final byte[] CONJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
-            UTF8StringUtil.writeStringToBytes(CONJUNCTIVE_SEARCH_MODE_OPTION);
+            UTF8StringUtil.writeStringToBytes(SearchMode.ALL.getValue());
+
+    public enum SearchMode {
+        ANY("any"),
+        ALL("all");
+
+        private String value;
+
+        SearchMode(String value) {
+            this.value = value;
+        }
+
+        public String getValue() {
+            return value;
+        }
+    }
+
+    public static final String FULLTEXT_CONFIG_OPTION = "config";
+    private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
 
     static {
         paramTypeMap.put(SEARCH_MODE_OPTION, ATypeTag.STRING);
+        paramTypeMap.put(FULLTEXT_CONFIG_OPTION, ATypeTag.STRING);
     }
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new FullTextContainsDescriptor();
+            return new FullTextContainsFunctionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FullTextContainsTypeInferer();
         }
     };
 
+    public FullTextContainsFunctionDescriptor() {
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        super.setImmutableStates(states);
+
+        IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory = (IFullTextConfigEvaluatorFactory) states[0];
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
+    }
+
     /**
      * Creates full-text search evaluator. There are three arguments:
      * arg0: Expression1 - search field
@@ -75,11 +115,11 @@ public class FullTextContainsDescriptor extends AbstractScalarFunctionDynamicDes
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
             throws AlgebricksException {
         return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
+            private static final long serialVersionUID = 2L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new FullTextContainsEvaluator(args, ctx);
+                return new FullTextContainsFunctionEvaluator(args, ctx, fullTextConfigEvaluatorFactory);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java
similarity index 52%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java
index a9197f2..3263656 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java
@@ -23,47 +23,33 @@ import org.apache.asterix.common.annotations.MissingNullInOutFunction;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.FullTextContainsEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
+// Descriptor for the ftcontains() function.
+//
+// This is different from the full-text config descriptor (FullTextConfigDescriptor)
+// which contains a tokenizer and full-text filter descriptors (AbstractFullTextFilterDescriptor).
 @MissingNullInOutFunction
-public class FullTextContainsWithoutOptionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new FullTextContainsWithoutOptionDescriptor();
-        }
-    };
-
-    /**
-     * Creates full-text search evaluator. There are two arguments:
-     * arg0: Expression1 - search field
-     * arg1: Expression2 - search predicate
-     */
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
-            throws AlgebricksException {
-        return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new FullTextContainsEvaluator(args, ctx);
-            }
-        };
-    }
+public class FullTextContainsWithoutOptionFunctionDescriptor extends FullTextContainsFunctionDescriptor {
+    private static final long serialVersionUID = 2L;
 
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION;
     }
 
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new FullTextContainsWithoutOptionFunctionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new FunctionTypeInferers.FullTextContainsTypeInferer();
+        }
+    };
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
similarity index 57%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
index 6a7da02..0179c34 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
@@ -17,22 +17,25 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.asterix.runtime.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import org.apache.asterix.common.metadata.DataverseName;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+public abstract class AbstractFullTextFilterDescriptor implements IFullTextFilterDescriptor {
+    protected final DataverseName dataverseName;
+    protected final String name;
 
-    public boolean hasNext();
+    public AbstractFullTextFilterDescriptor(DataverseName dataverseName, String name) {
+        this.dataverseName = dataverseName;
+        this.name = name;
+    }
 
-    public void next();
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
 
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+    @Override
+    public String getName() {
+        return name;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java
new file mode 100644
index 0000000..6df92a1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+// Full-text config that contains a tokenizer (e.g. a WORK tokenizer) and multiple full-text filters (e.g. stopwords filter)
+// to tokenize and process tokens of full-text documents
+// When running the ftcontains() function, the full-text config can be used with or without a full-text index
+public class FullTextConfigDescriptor implements IFullTextConfigDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final DataverseName dataverseName;
+    private final String name;
+    private final TokenizerCategory tokenizerCategory;
+    private final ImmutableList<String> filterNames;
+
+    public FullTextConfigDescriptor(DataverseName dataverseName, String name, TokenizerCategory tokenizerCategory,
+            ImmutableList<String> filterNames) {
+        this.dataverseName = dataverseName;
+        this.name = name;
+        this.tokenizerCategory = tokenizerCategory;
+        this.filterNames = filterNames;
+    }
+
+    // This built-in default full-text config will be used only when no full-text config is specified by the user.
+    // Note that the default ft config descriptor is not stored in metadata catalog,
+    // and if we are trying to get a full-text config descriptor with a name of null or empty string,
+    // the metadata manager will return this default full-text config without looking into the metadata catalog
+    // In this way we avoid the edge cases to insert or delete the default config in the metadata catalog
+    public static FullTextConfigDescriptor getDefaultFullTextConfig() {
+        return new FullTextConfigDescriptor(null, null, TokenizerCategory.WORD, ImmutableList.of());
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    // We need to exclude the full-text filter descriptors from the full-text config because both of them
+    // would be in the metadata cache, that means they should be immutable to guarantee consistency
+    // So we decide to let the caller to be responsible for fetching the filter descriptors from metadata,
+    // and pass the filters as an argument here
+    //
+    // Use the util function org.apache.asterix.metadata.utils.FullTextUtil.fetchFilterAndCreateConfigEvaluator()
+    // to fetch filters according to the filter names and create full-text config evaluator
+    @Override
+    public IFullTextConfigEvaluatorFactory createEvaluatorFactory(
+            ImmutableList<AbstractFullTextFilterDescriptor> filterDescriptors) {
+        ImmutableList.Builder<IFullTextFilterEvaluatorFactory> filtersBuilder = new ImmutableList.Builder<>();
+        for (IFullTextFilterDescriptor filterDescriptor : filterDescriptors) {
+            filtersBuilder.add(filterDescriptor.createEvaluatorFactory());
+        }
+
+        return new FullTextConfigEvaluatorFactory(name, tokenizerCategory, filtersBuilder.build());
+    }
+
+    @Override
+    public TokenizerCategory getTokenizerCategory() {
+        return tokenizerCategory;
+    }
+
+    @Override
+    public ImmutableList<String> getFilterNames() {
+        return filterNames;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java
similarity index 56%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java
index 6a7da02..97f00c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java
@@ -17,22 +17,22 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.asterix.runtime.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import java.io.Serializable;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 
-    public boolean hasNext();
+import com.google.common.collect.ImmutableList;
 
-    public void next();
+public interface IFullTextConfigDescriptor extends Serializable {
+    String getName();
 
-    public void reset(byte[] data, int start, int length);
+    TokenizerCategory getTokenizerCategory();
 
-    // Get the total number of tokens
-    public short getTokensCount();
+    ImmutableList<String> getFilterNames();
 
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+    IFullTextConfigEvaluatorFactory createEvaluatorFactory(
+            ImmutableList<AbstractFullTextFilterDescriptor> filterDescriptors);
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java
new file mode 100644
index 0000000..630c8e8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+
+// Full-text filter descriptor that contains all the necessary information in the compile-time.
+// After compilation, a filter evaluator factory will be created at run-time,
+// and then a filter evaluator will be produced by the evaluator factory to process tokens after tokenization.
+//
+// Note that the filter concepts (descriptor, evaluator factory and evaluator)
+// are wrapped in the full-text config concepts accordingly.
+// The design of the filter is never to be called directly but via the full-text config.
+public interface IFullTextFilterDescriptor extends Serializable {
+    String getName();
+
+    FullTextFilterType getFilterType();
+
+    IFullTextFilterEvaluatorFactory createEvaluatorFactory();
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java
new file mode 100644
index 0000000..1dbef70
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import java.util.List;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.StopwordsFullTextFilterEvaluatorFactory;
+
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterDescriptor extends AbstractFullTextFilterDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public ImmutableList<String> stopwordList;
+
+    public StopwordsFullTextFilterDescriptor(DataverseName dataverseName, String name,
+            ImmutableList<String> stopwordList) {
+        super(dataverseName, name);
+        this.stopwordList = stopwordList;
+    }
+
+    @Override
+    public FullTextFilterType getFilterType() {
+        return FullTextFilterType.STOPWORDS;
+    }
+
+    public List<String> getStopwordList() {
+        return this.stopwordList;
+    }
+
+    @Override
+    public IFullTextFilterEvaluatorFactory createEvaluatorFactory() {
+        return new StopwordsFullTextFilterEvaluatorFactory(name, stopwordList);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 42e27da..17159d3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -334,8 +334,8 @@ import org.apache.asterix.runtime.evaluators.functions.CreateUUIDDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.DecodeDataverseDisplayNameDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.DecodeDataverseNameDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionFunctionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetJobParameterByNameDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetTypeDescriptor;
@@ -1081,8 +1081,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(LineRectanglePolygonAccessor.FACTORY);
 
         // full-text function
-        fc.add(FullTextContainsDescriptor.FACTORY);
-        fc.add(FullTextContainsWithoutOptionDescriptor.FACTORY);
+        fc.add(FullTextContainsFunctionDescriptor.FACTORY);
+        fc.add(FullTextContainsWithoutOptionFunctionDescriptor.FACTORY);
 
         // Record functions.
         fc.add(GetRecordFieldsDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index ebb0717..40c6944 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -321,6 +321,16 @@ public final class FunctionTypeInferers {
         }
     }
 
+    public static final class FullTextContainsTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            // get the full-text config evaluator from the expr which is set in FullTextContainsParameterCheckAndSetRule
+            fd.setImmutableStates(funcExpr.getOpaqueParameters()[0]);
+        }
+    }
+
     private static IAType[] getArgumentsTypes(AbstractFunctionCallExpression funExp, IVariableTypeEnvironment ctx)
             throws AlgebricksException {
         IAType[] argsTypes = new IAType[funExp.getArguments().size()];
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
index 366c2a7..23d81dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -96,5 +96,14 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+  <dependency>
+    <groupId>org.apache.commons</groupId>
+    <artifactId>commons-lang3</artifactId>
+  </dependency>
+  <dependency>
+    <groupId>com.google.guava</groupId>
+    <artifactId>guava</artifactId>
+  </dependency>
+
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index 3ed47e6..4477e65 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -27,6 +27,9 @@ import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -34,6 +37,7 @@ public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOpe
     private static final long serialVersionUID = 1L;
 
     private final IBinaryTokenizerFactory tokenizerFactory;
+    private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     // Field that will be tokenized.
     private final int docField;
     // operator will append these key fields to each token, e.g., as
@@ -53,10 +57,12 @@ public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOpe
     private final IMissingWriterFactory missingWriterFactory;
 
     public BinaryTokenizerOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IBinaryTokenizerFactory tokenizerFactory, int docField, int[] keyFields, boolean addNumTokensKey,
-            boolean writeKeyFieldsFirst, boolean writeMissing, IMissingWriterFactory missingWriterFactory) {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            int docField, int[] keyFields, boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
+            IMissingWriterFactory missingWriterFactory) {
         super(spec, 1, 1);
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.docField = docField;
         this.keyFields = keyFields;
         this.addNumTokensKey = addNumTokensKey;
@@ -69,9 +75,14 @@ public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOpe
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+        IFullTextConfigEvaluator fullTextConfigEvaluator =
+                fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
         return new BinaryTokenizerOperatorNodePushable(ctx,
-                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), outRecDescs[0],
-                tokenizerFactory.createTokenizer(), docField, keyFields, addNumTokensKey, writeKeyFieldsFirst,
-                writeMissing, missingWriterFactory);
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), outRecDescs[0], tokenizer,
+                fullTextConfigEvaluator, docField, keyFields, addNumTokensKey, writeKeyFieldsFirst, writeMissing,
+                missingWriterFactory);
     }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 3df185a..b9c8c8a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -36,13 +36,14 @@ import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
 public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
 
     private final IHyracksTaskContext ctx;
-    private final IBinaryTokenizer tokenizer;
+    private final IFullTextConfigEvaluator fullTextEvaluator;
     private final int docField;
     private final int[] keyFields;
     private final boolean addNumTokensKey;
@@ -59,11 +60,24 @@ public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnary
     private FrameTupleAppender appender;
 
     public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
-            RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int docField, int[] keyFields,
-            boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
+            RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, IFullTextConfigEvaluator fullTextEvaluator,
+            int docField, int[] keyFields, boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
             IMissingWriterFactory missingWriterFactory) {
         this.ctx = ctx;
-        this.tokenizer = tokenizer;
+        this.fullTextEvaluator = fullTextEvaluator;
+        // Need to use the tokenizer created in the upper-layer when:
+        // 1. The tokenizer is of TokenizerCategory.NGram rather than Word
+        // 2. If the tokenizer is a TokenizerCategory.Word one, then its parameters
+        //    (e.g. boolean ignoreTokenCount, boolean sourceHasTypeTag) may be different
+        //    from the tokenizer in the default full-text config.
+        //
+        //    Note that those parameters might be call-site specific, for example, one string byte array may contains
+        //    the ATypeTag.String in it while some doesn't. Even though the tokenizers are both Word tokenizer,
+        //    we still need to set different tokenizer here.
+        //    The different tokeniers are defined in BinaryTokenizerFactoryProvider.
+        //    The big plan is to remove the tokenizer from this interface and use fullTextEvaluator only.
+        this.fullTextEvaluator.setTokenizer(tokenizer);
+
         this.docField = docField;
         this.keyFields = keyFields;
         this.addNumTokensKey = addNumTokensKey;
@@ -91,20 +105,20 @@ public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnary
         for (int i = 0; i < tupleCount; i++) {
             tuple.reset(accessor, i);
 
-            short numTokens = 0;
+            int numTokens = 0;
 
             if (!isDocFieldMissing(tuple)) {
-                tokenizer.reset(tuple.getFieldData(docField), tuple.getFieldStart(docField),
+                fullTextEvaluator.reset(tuple.getFieldData(docField), tuple.getFieldStart(docField),
                         tuple.getFieldLength(docField));
                 if (addNumTokensKey) {
                     // Get the total number of tokens.
-                    numTokens = tokenizer.getTokensCount();
+                    numTokens = fullTextEvaluator.getTokensCount();
                 }
                 // Write token and data into frame by following the order specified
                 // in the writeKeyFieldsFirst field.
-                while (tokenizer.hasNext()) {
-                    tokenizer.next();
-                    IToken token = tokenizer.getToken();
+                while (fullTextEvaluator.hasNext()) {
+                    fullTextEvaluator.next();
+                    IToken token = fullTextEvaluator.getToken();
                     writeTuple(token, numTokens, i);
                 }
             } else if (writeMissing) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 0904c4a..0af1f3f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -42,6 +42,8 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
 import org.apache.hyracks.storage.common.IStorageManager;
@@ -54,33 +56,50 @@ import com.fasterxml.jackson.databind.node.ObjectNode;
 public class LSMInvertedIndexLocalResource extends LsmResource {
 
     private static final long serialVersionUID = 1L;
+    public static final String TOKEN_TYPE_TRAITS = "tokenTypeTraits";
+    public static final String TOKEN_CMP_FACTORIES = "tokenCmpFactories";
+    public static final String IS_PARTITIONED = "isPartitioned";
+    public static final String FULL_TEXT_CONFIG_EVALUATOR_FACTORY = "fullTextConfigEvaluatorFactory";
 
     private final ITypeTraits[] tokenTypeTraits;
     private final IBinaryComparatorFactory[] tokenCmpFactories;
+    // ToDo: totally replace tokenizer with fullTextConfig
+    // Currently, the configuration of tokenizer is complicated and the parameters
+    // (e.g. whether the string contains a ATypeTag.AString at the beginning),
+    // as a first step, we set the tokenizer of the full-text config on-the-fly
+    // See comments in FullTextConfigEvaluator
     private final IBinaryTokenizerFactory tokenizerFactory;
+    private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     private final boolean isPartitioned;
     private final int[] invertedIndexFields;
     private final int[] filterFieldsForNonBulkLoadOps;
     private final int[] invertedIndexFieldsForNonBulkLoadOps;
     private final double bloomFilterFalsePositiveRate;
 
-    public LSMInvertedIndexLocalResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
+    public LSMInvertedIndexLocalResource(
+            // inherited fields
+            String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            Map<String, String> mergePolicyProperties, boolean durable,
+            // new fields
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
             int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.isPartitioned = isPartitioned;
         this.invertedIndexFields = invertedIndexFields;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -90,13 +109,15 @@ public class LSMInvertedIndexLocalResource extends LsmResource {
 
     private LSMInvertedIndexLocalResource(IPersistedResourceRegistry registry, JsonNode json,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, boolean isPartitioned, int[] invertedIndexFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
-            double bloomFilterFalsePositiveRate) throws HyracksDataException {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
+            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate)
+            throws HyracksDataException {
         super(registry, json);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.isPartitioned = isPartitioned;
         this.invertedIndexFields = invertedIndexFields;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -116,19 +137,20 @@ public class LSMInvertedIndexLocalResource extends LsmResource {
         pageWriteCallbackFactory.initialize(serviceCtx, this);
         if (isPartitioned) {
             return InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits,
-                    cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache,
-                    file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
+                    bufferCache, file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
                     metadataPageManagerFactory, serviceCtx.getTracer());
         } else {
             return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
-                    tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache, file.getAbsolutePath(),
-                    bloomFilterFalsePositiveRate, mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this),
-                    ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits,
-                    filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                    invertedIndexFieldsForNonBulkLoadOps, durable, metadataPageManagerFactory, serviceCtx.getTracer());
+                    tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
+                    file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
+                    pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                    filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
+                    metadataPageManagerFactory, serviceCtx.getTracer());
         }
     }
 
@@ -140,14 +162,15 @@ public class LSMInvertedIndexLocalResource extends LsmResource {
         for (ITypeTraits tt : tokenTypeTraits) {
             tokenTypeTraitsArray.add(tt.toJson(registry));
         }
-        jsonObject.set("tokenTypeTraits", tokenTypeTraitsArray);
+        jsonObject.set(TOKEN_TYPE_TRAITS, tokenTypeTraitsArray);
         final ArrayNode tokenCmpFactoriesArray = OBJECT_MAPPER.createArrayNode();
         for (IBinaryComparatorFactory factory : tokenCmpFactories) {
             tokenCmpFactoriesArray.add(factory.toJson(registry));
         }
-        jsonObject.set("tokenCmpFactories", tokenCmpFactoriesArray);
+        jsonObject.set(TOKEN_CMP_FACTORIES, tokenCmpFactoriesArray);
         jsonObject.set("tokenizerFactory", tokenizerFactory.toJson(registry));
-        jsonObject.put("isPartitioned", isPartitioned);
+        jsonObject.set(FULL_TEXT_CONFIG_EVALUATOR_FACTORY, fullTextConfigEvaluatorFactory.toJson(registry));
+        jsonObject.put(IS_PARTITIONED, isPartitioned);
         jsonObject.putPOJO("invertedIndexFields", invertedIndexFields);
         jsonObject.putPOJO("filterFieldsForNonBulkLoadOps", filterFieldsForNonBulkLoadOps);
         jsonObject.putPOJO("invertedIndexFieldsForNonBulkLoadOps", invertedIndexFieldsForNonBulkLoadOps);
@@ -173,7 +196,17 @@ public class LSMInvertedIndexLocalResource extends LsmResource {
                 tokenCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
         final IBinaryTokenizerFactory tokenizerFactory =
                 (IBinaryTokenizerFactory) registry.deserialize(json.get("tokenizerFactory"));
-        final boolean isPartitioned = json.get("isPartitioned").asBoolean();
+
+        final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
+        // back-compatible: the local resource in an older version of AsterixDB may not contain the newly added fullTextConfigFactory
+        if (json.has(FULL_TEXT_CONFIG_EVALUATOR_FACTORY)) {
+            fullTextConfigEvaluatorFactory = (IFullTextConfigEvaluatorFactory) registry
+                    .deserialize(json.get(FULL_TEXT_CONFIG_EVALUATOR_FACTORY));
+        } else {
+            fullTextConfigEvaluatorFactory = FullTextConfigEvaluatorFactory.getDefaultFactory();
+        }
+
+        final boolean isPartitioned = json.get(IS_PARTITIONED).asBoolean();
         final int[] invertedIndexFields = OBJECT_MAPPER.convertValue(json.get("invertedIndexFields"), int[].class);
         final int[] filterFieldsForNonBulkLoadOps =
                 OBJECT_MAPPER.convertValue(json.get("filterFieldsForNonBulkLoadOps"), int[].class);
@@ -181,7 +214,7 @@ public class LSMInvertedIndexLocalResource extends LsmResource {
                 OBJECT_MAPPER.convertValue(json.get("invertedIndexFieldsForNonBulkLoadOps"), int[].class);
         final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
         return new LSMInvertedIndexLocalResource(registry, json, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
-                bloomFilterFalsePositiveRate);
+                fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index c36e8d3..09c9699 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -31,6 +31,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.IResource;
 import org.apache.hyracks.storage.common.IStorageManager;
@@ -46,16 +47,20 @@ public class LSMInvertedIndexLocalResourceFactory extends LsmResourceFactory {
     private final int[] filterFieldsForNonBulkLoadOps;
     private final int[] invertedIndexFieldsForNonBulkLoadOps;
     private final double bloomFilterFalsePositiveRate;
+    private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
 
     public LSMInvertedIndexLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
+            // Inherited fields
             IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            Map<String, String> mergePolicyProperties, boolean durable,
+            // New fields
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
             int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
@@ -63,7 +68,9 @@ public class LSMInvertedIndexLocalResourceFactory extends LsmResourceFactory {
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
+        // ToDo: totally replace tokenizerFactory with full-text config
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.isPartitioned = isPartitioned;
         this.invertedIndexFields = invertedIndexFields;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -77,8 +84,8 @@ public class LSMInvertedIndexLocalResourceFactory extends LsmResourceFactory {
                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory, isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+                tokenizerFactory, fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 13a649d..2cee54d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -31,6 +31,7 @@ import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -44,6 +45,7 @@ public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractSingleActi
     private final boolean isFullTextSearchQuery;
     private final IIndexDataflowHelperFactory indexHelperFactory;
     private final IBinaryTokenizerFactory queryTokenizerFactory;
+    private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     private final boolean retainInput;
     private final boolean retainMissing;
     private final IMissingWriterFactory missingWriterFactory;
@@ -54,14 +56,16 @@ public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractSingleActi
 
     public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int queryField, IIndexDataflowHelperFactory indexHelperFactory,
-            IBinaryTokenizerFactory queryTokenizerFactory, IInvertedIndexSearchModifierFactory searchModifierFactory,
-            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
-            ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
-            int frameLimit) {
+            IBinaryTokenizerFactory queryTokenizerFactory,
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            IInvertedIndexSearchModifierFactory searchModifierFactory, boolean retainInput, boolean retainMissing,
+            IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields,
+            boolean appendIndexFilter, int frameLimit) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.queryTokenizerFactory = queryTokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.queryField = queryField;
         this.retainInput = retainInput;
         this.retainMissing = retainMissing;
@@ -84,7 +88,7 @@ public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractSingleActi
         return new LSMInvertedIndexSearchOperatorNodePushable(ctx,
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), partition, minFilterFieldIndexes,
                 maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, searchModifier, queryTokenizerFactory, queryField, isFullTextSearchQuery,
-                numOfFields, appendIndexFilter, frameLimit);
+                searchCallbackFactory, searchModifier, queryTokenizerFactory, fullTextConfigEvaluatorFactory,
+                queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, frameLimit);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index a27dea7..e4dbb43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -34,7 +34,10 @@ import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.ISearchPredicate;
@@ -43,6 +46,7 @@ public class LSMInvertedIndexSearchOperatorNodePushable extends IndexSearchOpera
 
     protected final IInvertedIndexSearchModifier searchModifier;
     protected final IBinaryTokenizerFactory binaryTokenizerFactory;
+    protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     protected final int queryFieldIndex;
     protected final int numOfFields;
     // Keeps the information whether the given query is a full-text search or not.
@@ -57,12 +61,14 @@ public class LSMInvertedIndexSearchOperatorNodePushable extends IndexSearchOpera
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             IInvertedIndexSearchModifier searchModifier, IBinaryTokenizerFactory binaryTokenizerFactory,
-            int queryFieldIndex, boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
-            int frameLimit) throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, int queryFieldIndex,
+            boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter, int frameLimit)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
         this.searchModifier = searchModifier;
         this.binaryTokenizerFactory = binaryTokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.queryFieldIndex = queryFieldIndex;
         this.isFullTextSearchQuery = isFullTextSearchQuery;
         // If retainInput is true, the frameTuple is created in IndexSearchOperatorNodePushable.open().
@@ -79,7 +85,11 @@ public class LSMInvertedIndexSearchOperatorNodePushable extends IndexSearchOpera
 
     @Override
     protected ISearchPredicate createSearchPredicate() {
-        return new InvertedIndexSearchPredicate(binaryTokenizerFactory.createTokenizer(), searchModifier, minFilterKey,
+        IBinaryTokenizer tokenizer = binaryTokenizerFactory.createTokenizer();
+        IFullTextConfigEvaluator fullTextConfigEvaluator =
+                fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
+        return new InvertedIndexSearchPredicate(tokenizer, fullTextConfigEvaluator, searchModifier, minFilterKey,
                 maxFilterKey, isFullTextSearchQuery);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java
index 6a7da02..6225101 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java
@@ -17,22 +17,23 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+public abstract class AbstractFullTextFilterEvaluator implements IFullTextFilterEvaluator {
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+    protected final String name;
+    protected final FullTextFilterType type;
 
-    public boolean hasNext();
+    public AbstractFullTextFilterEvaluator(String name, FullTextFilterType type) {
+        this.name = name;
+        this.type = type;
+    }
 
-    public void next();
+    public String getName() {
+        return name;
+    }
 
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+    public FullTextFilterType getFilterType() {
+        return type;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
similarity index 60%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
index 6a7da02..d883829 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
@@ -17,22 +17,17 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+public abstract class AbstractFullTextFilterEvaluatorFactory implements IFullTextFilterEvaluatorFactory {
+    protected final String name;
+    protected final FullTextFilterType type;
+    protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
-}
+    protected AbstractFullTextFilterEvaluatorFactory(String name, FullTextFilterType type) {
+        this.name = name;
+        this.type = type;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java
new file mode 100644
index 0000000..6805c43
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+import com.google.common.collect.ImmutableList;
+
+// FullTextConfigEvaluator is a run-time evaluator while the IFullTextConfigDescriptor is a compile-time descriptor
+//
+// The descriptor is responsible for serialization (i.e. distributed from the CC to NC)
+// and Metadata translator (i.e. be written to the metadata catalog)
+// And the analyzer is to process the tokens in each NC at run-time
+public class FullTextConfigEvaluator implements IFullTextConfigEvaluator {
+
+    // name is not necessary for run-time token processing, however, let's keep this field for debugging purpose
+    // The name of the evaluator is supposed to have the dataverse name and the name of the full-text config descriptor
+    private final String name;
+    // Due to the limitation of existing code (e.g. some call sites assume the input has a Asterix type tag while some don't),
+    // we may need to replace the tokenizer on-the-fly in certain call sites.
+    // So this field is not final
+    private IBinaryTokenizer tokenizer;
+    private final ImmutableList<IFullTextFilterEvaluator> filters;
+
+    private IToken currentToken;
+    private IToken nextToken;
+
+    public FullTextConfigEvaluator(String name, TokenizerCategory tokenizerCategory,
+            ImmutableList<IFullTextFilterEvaluator> filters) {
+        this.name = name;
+        this.filters = filters;
+
+        switch (tokenizerCategory) {
+            case WORD:
+                // Currently, the tokenizer will be set later after the analyzer created
+                // This is because the tokenizer logic is complex,
+                // and we are already using a dedicated tokenizer factory to create tokenizer.
+                // One tricky part of tokenizer is that it can be call-site specific, e.g. the string in some call-site
+                // has the ATypeTag.String in the beginning of its byte array, and some doesn't,
+                // so if we only know the category of the tokenizer, e.g. a WORD tokenizer,
+                // we still cannot create a suitable tokenizer here as the tokenizer factory does.
+                //
+                // Finally we should get rid of the dedicated tokenizer factory and put its related logic
+                // in the full-text descriptor and analyzer
+                this.tokenizer = null;
+                break;
+            case NGRAM:
+            default:
+                throw new IllegalArgumentException();
+        }
+    }
+
+    @Override
+    public IBinaryTokenizer getTokenizer() {
+        return tokenizer;
+    }
+
+    @Override
+    public TokenizerCategory getTokenizerCategory() {
+        return tokenizer.getTokenizerCategory();
+    }
+
+    @Override
+    public void setTokenizer(IBinaryTokenizer tokenizer) {
+        this.tokenizer = tokenizer;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length) {
+        currentToken = null;
+        nextToken = null;
+        tokenizer.reset(data, start, length);
+    }
+
+    @Override
+    public IToken getToken() {
+        return currentToken;
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (nextToken != null) {
+            return true;
+        }
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+            IToken candidateToken = tokenizer.getToken();
+            for (IFullTextFilterEvaluator filter : filters) {
+                // ToDo: Tokenizer of TokenizerType.List would return strings starting with the length,
+                // e.g. 8database where 8 is the length
+                // Should we let TokenizerType.List returns the same thing as TokenizerType.String to make things easier?
+                // If so, we need to remove the length in the input string in all the call site of the tokenizer
+                // Otherwise, filters need tokenizer.getTokenizerType to decide if they need to remove the length themselves
+                candidateToken = filter.processToken(tokenizer.getTokenizerType(), candidateToken);
+                // null means the token is removed, i.e. it is a stopword
+                if (candidateToken == null) {
+                    break;
+                }
+            }
+
+            if (candidateToken != null) {
+                nextToken = candidateToken;
+                break;
+            }
+        }
+
+        return nextToken != null;
+    }
+
+    @Override
+    public void next() {
+        currentToken = nextToken;
+        nextToken = null;
+    }
+
+    @Override
+    public int getTokensCount() {
+        return tokenizer.getTokensCount();
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java
new file mode 100644
index 0000000..cb7c86a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.ImmutableList;
+
+public class FullTextConfigEvaluatorFactory implements IFullTextConfigEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final String name;
+    private final TokenizerCategory tokenizerCategory;
+    private final ImmutableList<IFullTextFilterEvaluatorFactory> filters;
+
+    public FullTextConfigEvaluatorFactory(String name, TokenizerCategory tokenizerCategory,
+            ImmutableList<IFullTextFilterEvaluatorFactory> filters) {
+        this.name = name;
+        this.tokenizerCategory = tokenizerCategory;
+        this.filters = filters;
+    }
+
+    @Override
+    public IFullTextConfigEvaluator createFullTextConfigEvaluator() {
+        ImmutableList.Builder<IFullTextFilterEvaluator> filterEvaluatorsBuilder = ImmutableList.builder();
+        for (IFullTextFilterEvaluatorFactory factory : filters) {
+            filterEvaluatorsBuilder.add(factory.createFullTextFilterEvaluator());
+        }
+        return new FullTextConfigEvaluator(name, tokenizerCategory, filterEvaluatorsBuilder.build());
+    }
+
+    public static IFullTextConfigEvaluatorFactory getDefaultFactory() {
+        return new FullTextConfigEvaluatorFactory("default_config_evaluator_factory", TokenizerCategory.WORD,
+                ImmutableList.of());
+    }
+
+    private static final String FIELD_NAME = "name";
+    private static final String FIELD_TOKENIZER_CATEGORY = "tokenizerCategory";
+    private static final String FIELD_FILTERS = "filters";
+    protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put(FIELD_NAME, name);
+        json.put(FIELD_TOKENIZER_CATEGORY, tokenizerCategory.toString());
+
+        final ArrayNode filterArray = OBJECT_MAPPER.createArrayNode();
+        for (IFullTextFilterEvaluatorFactory filter : filters) {
+            filterArray.add(filter.toJson(registry));
+        }
+        json.set(FIELD_FILTERS, filterArray);
+
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final String name = json.get(FIELD_NAME).asText();
+        final String tokenizerCategoryStr = json.get(FIELD_TOKENIZER_CATEGORY).asText();
+        TokenizerCategory tc = TokenizerCategory.getEnumIgnoreCase(tokenizerCategoryStr);
+
+        ArrayNode filtersJsonNode = (ArrayNode) json.get(FIELD_FILTERS);
+        ImmutableList.Builder<IFullTextFilterEvaluatorFactory> filtersBuilder = ImmutableList.builder();
+        for (int i = 0; i < filtersJsonNode.size(); i++) {
+            filtersBuilder.add((IFullTextFilterEvaluatorFactory) registry.deserialize(filtersJsonNode.get(i)));
+        }
+        return new FullTextConfigEvaluatorFactory(name, tc, filtersBuilder.build());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java
similarity index 53%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java
index 6a7da02..abfd753 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java
@@ -16,23 +16,31 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+import org.apache.commons.lang3.EnumUtils;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+public enum FullTextFilterType {
+    STOPWORDS("Stopwords"),
+    SYNONYM("Synonym"),
+    STEMMER("Stemmer");
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+    private final String value;
 
-    public boolean hasNext();
+    FullTextFilterType(String value) {
+        this.value = value;
+    }
 
-    public void next();
+    public String getValue() {
+        return value;
+    }
 
-    public void reset(byte[] data, int start, int length);
+    public static FullTextFilterType getEnumIgnoreCase(String str) {
+        FullTextFilterType type = EnumUtils.getEnumIgnoreCase(FullTextFilterType.class, str);
 
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+        if (type == null) {
+            throw new IllegalArgumentException("Cannot convert string " + str + " to FullTextFilterType!");
+        }
+        return type;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java
similarity index 51%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java
index 6a7da02..94ee693 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java
@@ -17,22 +17,32 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+public interface IFullTextConfigEvaluator {
+    String getName();
 
-    public boolean hasNext();
+    TokenizerCategory getTokenizerCategory();
 
-    public void next();
+    void setTokenizer(IBinaryTokenizer tokenizer);
 
-    public void reset(byte[] data, int start, int length);
+    void reset(byte[] data, int start, int length);
 
-    // Get the total number of tokens
-    public short getTokensCount();
+    // ToDo: wrap the tokenizer and filters into a dedicated Java class
+    // so that at runtime the operators (evaluators) don't touch the usedByIndices filed
+    // That means, the usedByIndices field should be modified via MetadataManager only at compile time
+    IBinaryTokenizer getTokenizer();
+
+    IToken getToken();
+
+    boolean hasNext();
 
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+    void next();
+
+    // Get the total number of tokens
+    // Currently, it returns the number of tokens in the original text, that means stopwords are still counted
+    int getTokensCount();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java
index 6a7da02..9469b66 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java
@@ -17,22 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import java.io.Serializable;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+import org.apache.hyracks.api.io.IJsonSerializable;
 
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+public interface IFullTextConfigEvaluatorFactory extends Serializable, IJsonSerializable {
+    IFullTextConfigEvaluator createFullTextConfigEvaluator();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java
similarity index 63%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java
index 6a7da02..9c36ea3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java
@@ -17,22 +17,14 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+// Full-text filter evaluator that process tokens
+// Such an evaluator is created via IFullTextFilterEvaluatorFactory,
+// and the run-time evaluator factory is created from IFullTextFilterDescriptor which is a compile-time concept.
+public interface IFullTextFilterEvaluator {
+    IToken processToken(TokenizerInfo.TokenizerType tokenizerType, IToken token);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java
similarity index 50%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java
index 6a7da02..7f77e0f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java
@@ -17,22 +17,19 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+import java.io.Serializable;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
+import org.apache.hyracks.api.io.IJsonSerializable;
 
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+// This full-text filter evaluator factory would to be stored in the index local resource,
+// so it needs to be IJsonSerializable.
+// Also, it would to be distributed from CC (compile-time) to NC (run-time), so it needs to be Serializable.
+//
+// Such a IFullTextFilterEvaluatorFactory should always be wrapped in a IFullTextConfigEvaluatorFactory
+// because filter cannot live without a config: a full-text config is responsible to tokenize strings
+// and then feed the tokens into the filters.
+public interface IFullTextFilterEvaluatorFactory extends IJsonSerializable, Serializable {
+    IFullTextFilterEvaluator createFullTextFilterEvaluator();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java
new file mode 100644
index 0000000..da5fbb0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import java.util.List;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterEvaluator extends AbstractFullTextFilterEvaluator {
+
+    private final ImmutableList<String> stopwordList;
+
+    public StopwordsFullTextFilterEvaluator(String name, ImmutableList<String> stopwordList) {
+        super(name, FullTextFilterType.STOPWORDS);
+        this.stopwordList = stopwordList;
+    }
+
+    public List<String> getStopwordList() {
+        return stopwordList;
+    }
+
+    @Override
+    public IToken processToken(TokenizerInfo.TokenizerType tokenizerType, IToken token) {
+        int start = token.getStartOffset();
+        int length = token.getTokenLength();
+
+        // The List tokenizer returns token starting with the token length,
+        // e.g. 8database where the byte of value 8 means the token has a length of 8
+        // We need to skip the length to fetch the pure string (e.g. "database" without 8)
+        if (tokenizerType == TokenizerInfo.TokenizerType.LIST) {
+            int numBytesToStoreLength = UTF8StringUtil
+                    .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(token.getData(), token.getStartOffset()));
+            start += numBytesToStoreLength;
+            length -= numBytesToStoreLength;
+        }
+
+        String str = UTF8StringUtil.getUTF8StringInArray(token.getData(), start, length);
+        if (stopwordList.contains(str)) {
+            return null;
+        }
+
+        return token;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java
new file mode 100644
index 0000000..b31aa17
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterEvaluatorFactory extends AbstractFullTextFilterEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final ImmutableList<String> stopwordList;
+    private static final String STOPWORDS_FILTER_NAME = "stopwordsFilterName";
+    private static final String STOPWORDS_LIST = "stopwordsList";
+
+    public StopwordsFullTextFilterEvaluatorFactory(String name, ImmutableList<String> stopwordList) {
+        super(name, FullTextFilterType.STOPWORDS);
+        this.stopwordList = stopwordList;
+    }
+
+    @Override
+    public IFullTextFilterEvaluator createFullTextFilterEvaluator() {
+        return new StopwordsFullTextFilterEvaluator(name, stopwordList);
+    }
+
+    // ToDo: extract the common logics to a dedicated helper or utilization class after more filters are implemented
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), this.serialVersionUID);
+        json.put(STOPWORDS_FILTER_NAME, name);
+
+        ArrayNode stopwordsArrayNode = OBJECT_MAPPER.createArrayNode();
+        for (String s : stopwordList) {
+            stopwordsArrayNode.add(s);
+        }
+        json.set(STOPWORDS_LIST, stopwordsArrayNode);
+
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final String name = json.get(STOPWORDS_FILTER_NAME).asText();
+
+        // ToDo: create a new function to extract a list from json
+        ImmutableList.Builder<String> stopwordsBuilder = ImmutableList.<String> builder();
+        JsonNode stopwordsArrayNode = json.get(STOPWORDS_LIST);
+        for (int i = 0; i < stopwordsArrayNode.size(); i++) {
+            stopwordsBuilder.add(stopwordsArrayNode.get(i).asText());
+        }
+        ImmutableList<String> stopwords = stopwordsBuilder.build();
+
+        return new StopwordsFullTextFilterEvaluatorFactory(name, stopwords);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java
index 6a7da02..9c9b1a1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java
@@ -16,23 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
 
-package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+import org.apache.commons.lang3.EnumUtils;
 
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
+public enum TokenizerCategory {
+    NGRAM,
+    WORD;
 
-public interface IBinaryTokenizer {
-    public IToken getToken();
-
-    public boolean hasNext();
-
-    public void next();
-
-    public void reset(byte[] data, int start, int length);
-
-    // Get the total number of tokens
-    public short getTokensCount();
-
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+    public static TokenizerCategory getEnumIgnoreCase(String str) {
+        return EnumUtils.getEnumIgnoreCase(TokenizerCategory.class, str);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 068df9a..1469304 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -60,6 +60,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationC
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
@@ -82,6 +83,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
     private static final Logger LOGGER = LogManager.getLogger();
 
     protected final IBinaryTokenizerFactory tokenizerFactory;
+    protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
 
     // Type traits and comparators for tokens and inverted-list elements.
     protected final ITypeTraits[] invListTypeTraits;
@@ -97,16 +99,17 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            ITracer tracer) throws HyracksDataException {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+            int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
                 invertedIndexFields, tracer);
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
@@ -406,6 +409,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             VirtualFreePageManager virtualFreePageManager, int id) throws HyracksDataException {
         return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                fullTextConfigEvaluatorFactory,
                 ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 23d6221..de1fc49 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -37,6 +37,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -51,16 +52,16 @@ public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
             double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            ITracer tracer) throws HyracksDataException {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+            int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
                 bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
+                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
+                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     @Override
@@ -68,7 +69,8 @@ public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
             VirtualFreePageManager virtualFreePageManager, int id) throws HyracksDataException {
         return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
                 virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory, ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+                tokenizerFactory, fullTextConfigEvaluatorFactory,
+                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 914a469..5d74529 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -35,6 +35,7 @@ import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -50,6 +51,7 @@ public class InMemoryInvertedIndex implements IInPlaceInvertedIndex {
     protected final ITypeTraits[] invListTypeTraits;
     protected final IBinaryComparatorFactory[] invListCmpFactories;
     protected final IBinaryTokenizerFactory tokenizerFactory;
+    protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
 
     protected final ITypeTraits[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeCmpFactories;
@@ -57,12 +59,14 @@ public class InMemoryInvertedIndex implements IInPlaceInvertedIndex {
     public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IPageManager virtualFreePageManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws HyracksDataException {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            FileReference btreeFileRef) throws HyracksDataException {
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
         // BTree tuples: <tokens, inverted-list elements>.
         int numBTreeFields = tokenTypeTraits.length + invListTypeTraits.length;
         btreeTypeTraits = new ITypeTraits[numBTreeFields];
@@ -174,7 +178,8 @@ public class InMemoryInvertedIndex implements IInPlaceInvertedIndex {
     @Override
     public InMemoryInvertedIndexAccessor createAccessor(IIndexAccessParameters iap) throws HyracksDataException {
         return new InMemoryInvertedIndexAccessor(this,
-                new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory),
+                new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory,
+                        fullTextConfigEvaluatorFactory),
                 (IHyracksTaskContext) iap.getParameters().get(HyracksConstants.HYRACKS_TASK_CONTEXT));
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
index 0457b46..1a52d2e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -27,6 +27,8 @@ import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexTokenizingTupleIterator;
@@ -46,14 +48,16 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
 
     // To generate in-memory BTree tuples for insertions.
     private final IBinaryTokenizerFactory tokenizerFactory;
+    private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
     private InvertedIndexTokenizingTupleIterator tupleIter;
     private boolean destroyed = false;
 
     InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory) {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) {
         this.btree = btree;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
+        this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
     }
 
     @Override
@@ -93,9 +97,11 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
     }
 
     protected void setTokenizingTupleIterator() {
-        IBinaryTokenizer tokenizer = getTokenizerFactory().createTokenizer();
+        IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+        IFullTextConfigEvaluator fullTextConfigEvaluator =
+                fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
         tupleIter = new InvertedIndexTokenizingTupleIterator(tokenCmpFactories.length,
-                btree.getFieldCount() - tokenCmpFactories.length, tokenizer);
+                btree.getFieldCount() - tokenCmpFactories.length, tokenizer, fullTextConfigEvaluator);
     }
 
     public InvertedIndexTokenizingTupleIterator getTupleIter() {
@@ -122,6 +128,10 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
         return tokenizerFactory;
     }
 
+    public IFullTextConfigEvaluatorFactory getFullTextConfigEvaluatorFactory() {
+        return fullTextConfigEvaluatorFactory;
+    }
+
     public void setTupleIter(InvertedIndexTokenizingTupleIterator tupleIter) {
         this.tupleIter = tupleIter;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index 806bbf5..bd9ce60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -31,6 +31,7 @@ import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -47,9 +48,10 @@ public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex impl
     public PartitionedInMemoryInvertedIndex(IBufferCache memBufferCache, IPageManager memFreePageManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws HyracksDataException {
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+            FileReference btreeFileRef) throws HyracksDataException {
         super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, tokenizerFactory, btreeFileRef);
+                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
     }
 
     @Override
@@ -86,8 +88,8 @@ public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex impl
     @Override
     public PartitionedInMemoryInvertedIndexAccessor createAccessor(IIndexAccessParameters iap)
             throws HyracksDataException {
-        return new PartitionedInMemoryInvertedIndexAccessor(this,
-                new PartitionedInMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory), iap);
+        return new PartitionedInMemoryInvertedIndexAccessor(this, new PartitionedInMemoryInvertedIndexOpContext(btree,
+                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory), iap);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
index 9cfaf7a..6b6401a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
@@ -21,6 +21,8 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.inmemory;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedIndexTokenizingTupleIterator;
@@ -28,13 +30,15 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedI
 public class PartitionedInMemoryInvertedIndexOpContext extends InMemoryInvertedIndexOpContext {
 
     public PartitionedInMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory) {
-        super(btree, tokenCmpFactories, tokenizerFactory);
+            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) {
+        super(btree, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory);
     }
 
     protected void setTokenizingTupleIterator() {
         IBinaryTokenizer tokenizer = getTokenizerFactory().createTokenizer();
+        IFullTextConfigEvaluator fullTextConfigEvaluator =
+                getFullTextConfigEvaluatorFactory().createFullTextConfigEvaluator();
         setTupleIter(new PartitionedInvertedIndexTokenizingTupleIterator(tokenCmpFactories.length,
-                btree.getFieldCount() - tokenCmpFactories.length, tokenizer));
+                btree.getFieldCount() - tokenCmpFactories.length, tokenizer, fullTextConfigEvaluator));
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 1a08fd0..2fb620c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -45,8 +45,8 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearche
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -126,25 +126,27 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
     protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException {
         ITupleReference queryTuple = searchPred.getQueryTuple();
         int queryFieldIndex = searchPred.getQueryFieldIndex();
-        IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer();
+        IFullTextConfigEvaluator fullTextAnalyzer = searchPred.getFullTextConfigEvaluator();
+        fullTextAnalyzer.setTokenizer(searchPred.getQueryTokenizer());
+
         // Is this a full-text query?
         // Then, the last argument is conjunctive or disjunctive search option, not a query text.
         // Thus, we need to remove the last argument.
         boolean isFullTextSearchQuery = searchPred.getIsFullTextSearchQuery();
         // Get the type of query tokenizer.
-        TokenizerType queryTokenizerType = queryTokenizer.getTokenizerType();
+        TokenizerType queryTokenizerType = fullTextAnalyzer.getTokenizer().getTokenizerType();
         int tokenCountInOneField = 0;
 
         queryTokenAppender.reset(queryTokenFrame, true);
-        queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
+        fullTextAnalyzer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
                 queryTuple.getFieldLength(queryFieldIndex));
 
-        while (queryTokenizer.hasNext()) {
-            queryTokenizer.next();
+        while (fullTextAnalyzer.hasNext()) {
+            fullTextAnalyzer.next();
             queryTokenBuilder.reset();
             tokenCountInOneField++;
             try {
-                IToken token = queryTokenizer.getToken();
+                IToken token = fullTextAnalyzer.getToken();
                 // For the full-text search, we don't support a phrase search yet.
                 // So, each field should have only one token.
                 // If it's a list, it can have multiple keywords in it. But, each keyword should not be a phrase.
@@ -161,6 +163,7 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
                     }
                 }
 
+                // Includes the length of the string, e.g. 8database where 8 (of type byte instead of char) is the length of "database"
                 token.serializeToken(queryTokenBuilder.getFieldData());
                 queryTokenBuilder.addFieldEndOffset();
                 // WARNING: assuming one frame is big enough to hold all tokens
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
index 4ee685b..3cf9066 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
@@ -22,6 +22,7 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.search;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.impls.AbstractSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.common.MultiComparator;
 
@@ -31,21 +32,27 @@ public class InvertedIndexSearchPredicate extends AbstractSearchPredicate {
     private ITupleReference queryTuple;
     private int queryFieldIndex;
     private final IBinaryTokenizer queryTokenizer;
+    private final IFullTextConfigEvaluator fullTextConfigEvaluator;
     private final IInvertedIndexSearchModifier searchModifier;
     // Keeps the information whether the given query is a full-text search or not.
     // We need to have this information to stop the search process since we don't allow a phrase search yet.
     private boolean isFullTextSearchQuery;
 
-    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier) {
+    // Used for test only
+    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer,
+            IFullTextConfigEvaluator fullTextConfigEvaluator, IInvertedIndexSearchModifier searchModifier) {
         this.queryTokenizer = queryTokenizer;
+        this.fullTextConfigEvaluator = fullTextConfigEvaluator;
         this.searchModifier = searchModifier;
         this.isFullTextSearchQuery = false;
     }
 
-    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier,
+    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer,
+            IFullTextConfigEvaluator fullTextConfigEvaluator, IInvertedIndexSearchModifier searchModifier,
             ITupleReference minFilterTuple, ITupleReference maxFilterTuple, boolean isFullTextSearchQuery) {
         super(minFilterTuple, maxFilterTuple);
         this.queryTokenizer = queryTokenizer;
+        this.fullTextConfigEvaluator = fullTextConfigEvaluator;
         this.searchModifier = searchModifier;
         this.isFullTextSearchQuery = isFullTextSearchQuery;
     }
@@ -82,6 +89,10 @@ public class InvertedIndexSearchPredicate extends AbstractSearchPredicate {
         return queryTokenizer;
     }
 
+    public IFullTextConfigEvaluator getFullTextConfigEvaluator() {
+        return fullTextConfigEvaluator;
+    }
+
     @Override
     public MultiComparator getLowKeyComparator() {
         // TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
index 3a5224c..df06525 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
@@ -51,6 +52,8 @@ public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinary
         return byteIndex < sentenceEndOffset;
     }
 
+    // ToDo: current solution (where a white list defined for non-separator chars) is not suitable for double-surrogate utf-8 chars
+    // which are always judged as separators
     public static boolean isSeparator(char c) {
         return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER
                 || Character.getType(c) == Character.OTHER_NUMBER);
@@ -119,4 +122,9 @@ public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinary
     public TokenizerType getTokenizerType() {
         return TokenizerType.STRING;
     }
+
+    @Override
+    public TokenizerCategory getTokenizerCategory() {
+        return TokenizerCategory.WORD;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
index 6a7da02..0e7f30d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
@@ -19,20 +19,24 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
 
 public interface IBinaryTokenizer {
-    public IToken getToken();
+    IToken getToken();
 
-    public boolean hasNext();
+    boolean hasNext();
 
-    public void next();
+    void next();
 
-    public void reset(byte[] data, int start, int length);
+    void reset(byte[] data, int start, int length);
 
     // Get the total number of tokens
-    public short getTokensCount();
+    short getTokensCount();
 
-    // Get the tokenizer types
-    public TokenizerType getTokenizerType();
+    // Get the tokenizer types: String or List
+    TokenizerType getTokenizerType();
+
+    // WORD or NGRAM tokenizer
+    TokenizerCategory getTokenizerCategory();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
index 711a82f..0ab1662 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
@@ -136,4 +137,9 @@ public class NGramUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryToke
     public TokenizerType getTokenizerType() {
         return TokenizerType.STRING;
     }
+
+    @Override
+    public TokenizerCategory getTokenizerCategory() {
+        return TokenizerCategory.NGRAM;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
index 4cf0c55..19d6f11 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
@@ -25,6 +25,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
@@ -36,38 +37,43 @@ public class InvertedIndexTokenizingTupleIterator {
     protected final int invListFieldCount;
     protected final ArrayTupleBuilder tupleBuilder;
     protected final ArrayTupleReference tupleReference;
-    protected final IBinaryTokenizer tokenizer;
+    protected final IFullTextConfigEvaluator fullTextConfigEvaluator;
     protected ITupleReference inputTuple;
 
-    public InvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
-            IBinaryTokenizer tokenizer) {
+    public InvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount, IBinaryTokenizer tokenizer,
+            IFullTextConfigEvaluator fullTextConfigEvaluator) {
         this.invListFieldCount = invListFieldCount;
         this.tupleBuilder = new ArrayTupleBuilder(tokensFieldCount + invListFieldCount);
         this.tupleReference = new ArrayTupleReference();
-        this.tokenizer = tokenizer;
+        this.fullTextConfigEvaluator = fullTextConfigEvaluator;
+
+        // ToDo: check the codes in upper layer to see if we can remove tokenizer to use fullTextConfig instead
+        this.fullTextConfigEvaluator.setTokenizer(tokenizer);
     }
 
     public void reset(ITupleReference inputTuple) {
         this.inputTuple = inputTuple;
-        tokenizer.reset(inputTuple.getFieldData(DOC_FIELD_INDEX), inputTuple.getFieldStart(DOC_FIELD_INDEX),
-                inputTuple.getFieldLength(DOC_FIELD_INDEX));
+        fullTextConfigEvaluator.reset(inputTuple.getFieldData(DOC_FIELD_INDEX),
+                inputTuple.getFieldStart(DOC_FIELD_INDEX), inputTuple.getFieldLength(DOC_FIELD_INDEX));
     }
 
     public boolean hasNext() {
-        return tokenizer.hasNext();
+        return fullTextConfigEvaluator.hasNext();
     }
 
     public void next() throws HyracksDataException {
-        tokenizer.next();
-        IToken token = tokenizer.getToken();
+        fullTextConfigEvaluator.next();
+        IToken token = fullTextConfigEvaluator.getToken();
+
         tupleBuilder.reset();
-        // Add token field.
         try {
+            // Add token field.
             token.serializeToken(tupleBuilder.getFieldData());
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
         tupleBuilder.addFieldEndOffset();
+
         // Add inverted-list element fields.
         for (int i = 0; i < invListFieldCount; i++) {
             tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index a9a01e9..dcde832 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -52,6 +52,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
@@ -83,18 +84,21 @@ public class InvertedIndexUtils {
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
+            throws HyracksDataException {
         return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
     }
 
     public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
+            throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                fullTextConfigEvaluatorFactory, btreeFileRef);
     }
 
     public static OnDiskInvertedIndex createOnDiskInvertedIndex(IIOManager ioManager, IBufferCache bufferCache,
@@ -138,8 +142,9 @@ public class InvertedIndexUtils {
             List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, IBufferCache diskBufferCache,
+            String absoluteOnDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -179,17 +184,19 @@ public class InvertedIndexUtils {
 
         return new LSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
                 filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+                pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
             List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, IBufferCache diskBufferCache,
+            String absoluteOnDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -229,8 +236,9 @@ public class InvertedIndexUtils {
         return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper,
                 filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
-                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+                pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
index c44dffb..783afc6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
@@ -32,28 +33,33 @@ public class PartitionedInvertedIndexTokenizingTupleIterator extends InvertedInd
     protected short numTokens = 0;
 
     public PartitionedInvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
-            IBinaryTokenizer tokenizer) {
-        super(tokensFieldCount, invListFieldCount, tokenizer);
+            IBinaryTokenizer tokenizer, IFullTextConfigEvaluator fullTextConfigEvaluator) {
+        super(tokensFieldCount, invListFieldCount, tokenizer, fullTextConfigEvaluator);
     }
 
+    @Override
     public void reset(ITupleReference inputTuple) {
         super.reset(inputTuple);
         // Run through the tokenizer once to get the total number of tokens.
         numTokens = 0;
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
+        while (fullTextConfigEvaluator.hasNext()) {
+            fullTextConfigEvaluator.next();
             numTokens++;
         }
         super.reset(inputTuple);
     }
 
+    @Override
     public void next() throws HyracksDataException {
-        tokenizer.next();
-        IToken token = tokenizer.getToken();
+        fullTextConfigEvaluator.next();
+        IToken token = fullTextConfigEvaluator.getToken();
+
         tupleBuilder.reset();
         try {
             // Add token field.
             token.serializeToken(tupleBuilder.getFieldData());
+
+            // Different from super.next(): here we write the numTokens
             tupleBuilder.addFieldEndOffset();
             // Add field with number of tokens.
             tupleBuilder.getDataOutput().writeShort(numTokens);
@@ -61,6 +67,7 @@ public class PartitionedInvertedIndexTokenizingTupleIterator extends InvertedInd
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
+
         // Add inverted-list element fields.
         for (int i = 0; i < invListFieldCount; i++) {
             tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
index 4934660..aac8b50 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -116,6 +116,10 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 2fcd96f..9818147 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.storage.am.common.datagen.DataGenThread;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
@@ -90,6 +91,7 @@ public class LSMInvertedIndexTestWorker extends AbstractIndexTestWorker {
         IBinaryTokenizerFactory tokenizerFactory = invIndex.getTokenizerFactory();
         int searchModifierIndex = Math.abs(rnd.nextInt()) % TEST_SEARCH_MODIFIERS.length;
         InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizerFactory.createTokenizer(),
+                FullTextConfigEvaluatorFactory.getDefaultFactory().createFullTextConfigEvaluator(),
                 TEST_SEARCH_MODIFIERS[searchModifierIndex]);
 
         switch (op) {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 524904f..3e8db34 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -51,6 +51,7 @@ import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils.HyracksTaskTestContext;
 import org.apache.hyracks.storage.common.IIndex;
@@ -127,9 +128,10 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
 
     public static LSMInvertedIndexTestContext create(LSMInvertedIndexTestHarness harness,
             ISerializerDeserializer[] fieldSerdes, int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory,
-            InvertedIndexType invIndexType, int[] invertedIndexFields, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps) throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, InvertedIndexType invIndexType,
+            int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+            int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps)
+            throws HyracksDataException {
         ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IOManager ioManager = harness.getIOManager();
         IBinaryComparatorFactory[] allCmpFactories =
@@ -157,7 +159,7 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
                 invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
                 break;
             }
             case PARTITIONED_INMEMORY: {
@@ -165,7 +167,7 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
                         harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
                 break;
             }
             case ONDISK: {
@@ -183,8 +185,9 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
             case LSM: {
                 invIndex = InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
                         invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
-                        harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                        fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                         invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                         filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
@@ -196,9 +199,9 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
             case PARTITIONED_LSM: {
                 invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager,
                         harness.getVirtualBufferCaches(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
-                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                        harness.getOperationTracker(), harness.getIOScheduler(),
+                        tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
+                        harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+                        harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                         invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                         filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
@@ -217,7 +220,8 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
             case ONDISK:
             case LSM: {
                 indexTupleIter = new InvertedIndexTokenizingTupleIterator(invIndex.getTokenTypeTraits().length,
-                        invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer());
+                        invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer(),
+                        fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator());
                 break;
             }
             case PARTITIONED_INMEMORY:
@@ -225,7 +229,8 @@ public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
             case PARTITIONED_LSM: {
                 indexTupleIter =
                         new PartitionedInvertedIndexTokenizingTupleIterator(invIndex.getTokenTypeTraits().length,
-                                invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer());
+                                invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer(),
+                                fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator());
                 break;
             }
             default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index a5e47b5..3cc0913 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -88,6 +88,9 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccesso
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexMergeCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
@@ -113,11 +116,16 @@ import org.apache.hyracks.util.IThreadStats;
 import org.apache.hyracks.util.IThreadStatsCollector;
 import org.apache.hyracks.util.ThreadStats;
 
+import com.google.common.collect.ImmutableList;
+
 @SuppressWarnings("rawtypes")
 public class LSMInvertedIndexTestUtils {
 
     public static final int TEST_GRAM_LENGTH = 3;
 
+    public static FullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+            new FullTextConfigEvaluatorFactory("test_config", TokenizerCategory.WORD, ImmutableList.of());
+
     public static TupleGenerator createStringDocumentTupleGen(Random rnd) throws IOException {
         IFieldValueGenerator[] fieldGens = new IFieldValueGenerator[2];
         fieldGens[0] = new DocumentStringFieldValueGenerator(2, 10, 10000, rnd);
@@ -196,8 +204,9 @@ public class LSMInvertedIndexTestUtils {
         ITokenFactory tokenFactory = new UTF8WordTokenFactory();
         IBinaryTokenizerFactory tokenizerFactory =
                 new DelimitedUTF8StringBinaryTokenizerFactory(true, false, tokenFactory);
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
-                fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+        LSMInvertedIndexTestContext testCtx =
+                LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+                        fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
         return testCtx;
     }
 
@@ -207,8 +216,9 @@ public class LSMInvertedIndexTestUtils {
         ITokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
         IBinaryTokenizerFactory tokenizerFactory =
                 new DelimitedUTF8StringBinaryTokenizerFactory(true, false, tokenFactory);
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
-                fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+        LSMInvertedIndexTestContext testCtx =
+                LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+                        fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
         return testCtx;
     }
 
@@ -218,8 +228,9 @@ public class LSMInvertedIndexTestUtils {
         ITokenFactory tokenFactory = new UTF8NGramTokenFactory();
         IBinaryTokenizerFactory tokenizerFactory =
                 new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true, true, false, tokenFactory);
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
-                fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+        LSMInvertedIndexTestContext testCtx =
+                LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+                        fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
         return testCtx;
     }
 
@@ -229,8 +240,9 @@ public class LSMInvertedIndexTestUtils {
         ITokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
         IBinaryTokenizerFactory tokenizerFactory =
                 new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true, true, false, tokenFactory);
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
-                fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+        LSMInvertedIndexTestContext testCtx =
+                LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+                        fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
         return testCtx;
     }
 
@@ -561,7 +573,10 @@ public class LSMInvertedIndexTestUtils {
         iap.getParameters().put(HyracksConstants.HYRACKS_TASK_CONTEXT, ctx);
         IInvertedIndexAccessor accessor = (IInvertedIndexAccessor) invIndex.createAccessor(iap);
         IBinaryTokenizer tokenizer = testCtx.getTokenizerFactory().createTokenizer();
-        InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
+        IFullTextConfigEvaluator fullTextConfigEvaluator =
+                fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+        InvertedIndexSearchPredicate searchPred =
+                new InvertedIndexSearchPredicate(tokenizer, fullTextConfigEvaluator, searchModifier);
         List<ITupleReference> documentCorpus = testCtx.getDocumentCorpus();
         // Project away the primary-key field.
         int[] fieldPermutation = new int[] { 0 };
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index f96ed72..e83aedb 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -618,6 +618,10 @@ public class UTF8StringUtil {
         int count = 0;
 
         for (int i = 0; i < strlen; i++) {
+            // ToDo: we shouldn't use str.charAt(i) to convert raw byte array to UTF-8 chars
+            // one UTF-8 char has at most four bytes, and one Java char we get via str.charAt(i) has 2 bytes
+            // In this case, a UTF-8 char may be consistent of 2 Java chars, and 1 Java char can be converted into 3 UTF-8 bytes
+            // For the emoji, it can be 6 bytes after encoded to UTF-8
             c = str.charAt(i);
             utflen += UTF8StringUtil.getModifiedUTF8Len(c);
         }