You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wa...@apache.org on 2017/01/06 18:06:17 UTC

[4/4] asterixdb git commit: Full-text implementation step 3

Full-text implementation step 3

 - Implemented index-based full-text contains() function.
 - Fixed a bug in the full-text code that can't process join cases well.
 - Introduced full-text type index syntax. The functionality was already
   in the codebase but it was not being used. So, using this syntax, we
   can utilize the given index type - Not the length partitioned keyword index.
 - Added the support for SQL++
 - Default option will be set if a user doesn't provide any option.

Change-Id: I1087854ac7cf5b6ef5094e27a1646f12f6a8653f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1388
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <im...@apache.org>


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

Branch: refs/heads/master
Commit: c49405aaf0530f651b262c674acd11b1c7cc133d
Parents: d49bc6e
Author: Taewoo Kim <wa...@yahoo.com>
Authored: Wed Jan 4 00:30:43 2017 -0800
Committer: Taewoo Kim <wa...@yahoo.com>
Committed: Fri Jan 6 10:05:25 2017 -0800

----------------------------------------------------------------------
 .../physical/InvertedIndexPOperator.java        |   8 +-
 .../FullTextContainsParameterCheckRule.java     |  69 ++++++++++---
 .../am/AbstractIntroduceAccessMethodRule.java   |  53 +++++++---
 .../optimizer/rules/am/AccessMethodUtils.java   |  81 +++++++++++++--
 .../rules/am/InvertedIndexAccessMethod.java     | 102 +++++++++++++++++--
 .../rules/am/InvertedIndexJobGenParams.java     |  31 +++++-
 .../translator/util/FunctionCollection.java     |   2 +
 .../fulltext-contains-without-option.aql        |  55 ++++++++++
 .../inverted-index-basic/fulltext-contains.aql  |  55 ++++++++++
 .../fulltext-contains-without-option.plan       |  16 +++
 .../inverted-index-basic/fulltext-contains.plan |  16 +++
 .../fulltext/fulltext-01/fulltext-01.1.ddl.aql  |   3 +
 .../fulltext-01/fulltext-01.6.query.aql         |  25 +++++
 .../fulltext-01/fulltext-01.7.query.aql         |  26 +++++
 .../fulltext-01/fulltext-01.8.query.aql         |  26 +++++
 .../fulltext/fulltext-02/fulltext-02.1.ddl.aql  |   2 +
 .../fulltext-02/fulltext-02.7.query.aql         |  25 +++++
 .../fulltext-02/fulltext-02.8.query.aql         |  25 +++++
 .../fulltext/fulltext-03/fulltext-03.1.ddl.aql  |   3 +
 .../fulltext-03/fulltext-03.10.query.aql        |  26 +++++
 .../fulltext-03/fulltext-03.11.query.aql        |  25 +++++
 .../fulltext-03/fulltext-03.9.query.aql         |  26 +++++
 .../fulltext/fulltext-05/fulltext-05.1.ddl.aql  |   2 -
 .../fulltext/fulltext-06/fulltext-06.1.ddl.aql  |   2 -
 .../fulltext/fulltext-07/fulltext-07.1.ddl.aql  |   5 +-
 .../fulltext-07/fulltext-07.5.query.aql         |  26 +++++
 .../fulltext-index-01.1.ddl.aql                 |  55 ++++++++++
 .../fulltext-index-01.2.update.aql              |  24 +++++
 .../fulltext-index-01.3.query.aql               |  25 +++++
 .../fulltext-index-01.4.query.aql               |  26 +++++
 .../fulltext-index-01.5.query.aql               |  26 +++++
 .../fulltext-index-01.6.query.aql               |  25 +++++
 .../fulltext-index-01.7.query.aql               |  26 +++++
 .../fulltext-index-01.8.query.aql               |  26 +++++
 .../fulltext-index-02.1.ddl.aql                 |  59 +++++++++++
 .../fulltext-index-02.2.update.aql              |  24 +++++
 .../fulltext-index-02.3.query.aql               |  25 +++++
 .../fulltext-index-02.4.query.aql               |  25 +++++
 .../fulltext-index-02.5.query.aql               |  25 +++++
 .../fulltext-index-02.6.query.aql               |  25 +++++
 .../fulltext-index-02.7.query.aql               |  25 +++++
 .../fulltext-index-02.8.query.aql               |  25 +++++
 .../fulltext-index-03.1.ddl.aql                 |  72 +++++++++++++
 .../fulltext-index-03.10.query.aql              |  26 +++++
 .../fulltext-index-03.11.query.aql              |  25 +++++
 .../fulltext-index-03.2.update.aql              |  28 +++++
 .../fulltext-index-03.3.query.aql               |  26 +++++
 .../fulltext-index-03.4.query.aql               |  26 +++++
 .../fulltext-index-03.5.query.aql               |  26 +++++
 .../fulltext-index-03.6.query.aql               |  26 +++++
 .../fulltext-index-03.7.query.aql               |  25 +++++
 .../fulltext-index-03.8.query.aql               |  25 +++++
 .../fulltext-index-03.9.query.aql               |  26 +++++
 .../fulltext-index-04.1.ddl.aql                 |  58 +++++++++++
 .../fulltext-index-04.2.update.aql              |  19 ++++
 .../fulltext-index-04.3.query.aql               |  25 +++++
 .../fulltext-index-05.1.ddl.aql                 |  58 +++++++++++
 .../fulltext-index-05.2.update.aql              |  19 ++++
 .../fulltext-index-05.3.query.aql               |  25 +++++
 .../fulltext-index-06.1.ddl.aql                 |  58 +++++++++++
 .../fulltext-index-06.2.update.aql              |  19 ++++
 .../fulltext-index-06.3.query.aql               |  25 +++++
 .../fulltext-index-07.1.ddl.aql                 |  64 ++++++++++++
 .../fulltext-index-07.2.update.aql              |  28 +++++
 .../fulltext-index-07.3.query.aql               |  26 +++++
 .../fulltext-index-07.4.query.aql               |  26 +++++
 .../fulltext-index-07.5.query.aql               |  26 +++++
 .../fulltext-01/fulltext-01.1.ddl.sqlpp         |  53 ++++++++++
 .../fulltext-01/fulltext-01.2.update.sqlpp      |  24 +++++
 .../fulltext-01/fulltext-01.3.query.sqlpp       |  25 +++++
 .../fulltext-01/fulltext-01.4.query.sqlpp       |  25 +++++
 .../fulltext-01/fulltext-01.5.query.sqlpp       |  25 +++++
 .../fulltext-01/fulltext-01.6.query.sqlpp       |  25 +++++
 .../fulltext-01/fulltext-01.7.query.sqlpp       |  25 +++++
 .../fulltext-01/fulltext-01.8.query.sqlpp       |  25 +++++
 .../fulltext-02/fulltext-02.1.ddl.sqlpp         |  57 +++++++++++
 .../fulltext-02/fulltext-02.2.update.sqlpp      |  24 +++++
 .../fulltext-02/fulltext-02.3.query.sqlpp       |  25 +++++
 .../fulltext-02/fulltext-02.4.query.sqlpp       |  24 +++++
 .../fulltext-02/fulltext-02.5.query.sqlpp       |  25 +++++
 .../fulltext-02/fulltext-02.6.query.sqlpp       |  25 +++++
 .../fulltext-02/fulltext-02.7.query.sqlpp       |  25 +++++
 .../fulltext-02/fulltext-02.8.query.sqlpp       |  25 +++++
 .../fulltext-index-01.1.ddl.sqlpp               |  55 ++++++++++
 .../fulltext-index-01.2.update.sqlpp            |  24 +++++
 .../fulltext-index-01.3.query.sqlpp             |  25 +++++
 .../fulltext-index-01.4.query.sqlpp             |  25 +++++
 .../fulltext-index-01.5.query.sqlpp             |  25 +++++
 .../fulltext-index-01.6.query.sqlpp             |  25 +++++
 .../fulltext-index-01.7.query.sqlpp             |  25 +++++
 .../fulltext-index-01.8.query.sqlpp             |  25 +++++
 .../fulltext-index-02.1.ddl.sqlpp               |  59 +++++++++++
 .../fulltext-index-02.2.update.sqlpp            |  24 +++++
 .../fulltext-index-02.3.query.sqlpp             |  25 +++++
 .../fulltext-index-02.4.query.sqlpp             |  24 +++++
 .../fulltext-index-02.5.query.sqlpp             |  25 +++++
 .../fulltext-index-02.6.query.sqlpp             |  25 +++++
 .../fulltext-index-02.7.query.sqlpp             |  25 +++++
 .../fulltext-index-02.8.query.sqlpp             |  25 +++++
 .../fulltext/fulltext-01/fulltext-01.6.adm      |  20 ++++
 .../fulltext/fulltext-01/fulltext-01.7.adm      |  20 ++++
 .../fulltext/fulltext-01/fulltext-01.8.adm      |  20 ++++
 .../fulltext/fulltext-02/fulltext-02.7.adm      |  14 +++
 .../fulltext/fulltext-02/fulltext-02.8.adm      |  14 +++
 .../fulltext/fulltext-03/fulltext-03.10.adm     |  14 +++
 .../fulltext/fulltext-03/fulltext-03.11.adm     |  14 +++
 .../fulltext/fulltext-03/fulltext-03.9.adm      |  14 +++
 .../fulltext/fulltext-07/fulltext-07.3.adm      |  14 +++
 .../fulltext/fulltext-07/fulltext-07.4.adm      |  38 +++++++
 .../fulltext/fulltext-07/fulltext-07.5.adm      |  52 ++++++++++
 .../fulltext-index-01/fulltext-index-01.3.adm   |  20 ++++
 .../fulltext-index-01/fulltext-index-01.4.adm   |  20 ++++
 .../fulltext-index-01/fulltext-index-01.5.adm   |  20 ++++
 .../fulltext-index-01/fulltext-index-01.6.adm   |  20 ++++
 .../fulltext-index-01/fulltext-index-01.7.adm   |  20 ++++
 .../fulltext-index-01/fulltext-index-01.8.adm   |  20 ++++
 .../fulltext-index-02/fulltext-index-02.3.adm   |  38 +++++++
 .../fulltext-index-02/fulltext-index-02.4.adm   |  14 +++
 .../fulltext-index-02/fulltext-index-02.5.adm   |  38 +++++++
 .../fulltext-index-02/fulltext-index-02.6.adm   |  14 +++
 .../fulltext-index-02/fulltext-index-02.7.adm   |  14 +++
 .../fulltext-index-02/fulltext-index-02.8.adm   |  14 +++
 .../fulltext-index-03/fulltext-index-03.10.adm  |  14 +++
 .../fulltext-index-03/fulltext-index-03.11.adm  |  14 +++
 .../fulltext-index-03/fulltext-index-03.3.adm   |  38 +++++++
 .../fulltext-index-03/fulltext-index-03.4.adm   |  14 +++
 .../fulltext-index-03/fulltext-index-03.5.adm   |  38 +++++++
 .../fulltext-index-03/fulltext-index-03.6.adm   |  14 +++
 .../fulltext-index-03/fulltext-index-03.7.adm   |  38 +++++++
 .../fulltext-index-03/fulltext-index-03.8.adm   |  14 +++
 .../fulltext-index-03/fulltext-index-03.9.adm   |  14 +++
 .../fulltext-index-04/fulltext-index-04.3.adm   |   1 +
 .../fulltext-index-05/fulltext-index-05.3.adm   |   1 +
 .../fulltext-index-06/fulltext-index-06.3.adm   |   1 +
 .../fulltext-index-07/fulltext-index-07.3.adm   |  52 ++++++++++
 .../fulltext-index-07/fulltext-index-07.4.adm   |  52 ++++++++++
 .../fulltext-index-07/fulltext-index-07.5.adm   |  52 ++++++++++
 .../src/test/resources/runtimets/testsuite.xml  |  71 +++++++++++++
 .../resources/runtimets/testsuite_sqlpp.xml     |  22 ++++
 .../src/site/markdown/aql/fulltext.md           |  20 ++--
 .../asterix-doc/src/site/markdown/aql/manual.md |  12 ++-
 asterixdb/asterix-doc/src/site/site.xml         |   1 +
 .../asterix-lang-aql/src/main/javacc/AQL.jj     |   5 +
 .../lang/common/visitor/FormatPrintVisitor.java |   2 +
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj |   5 +
 .../common/AOrderedListBinaryTokenizer.java     |   6 ++
 .../asterix/om/functions/BuiltinFunctions.java  |   8 +-
 .../asterix/om/util/ConstantExpressionUtil.java |  19 +++-
 .../common/FullTextContainsEvaluator.java       |  90 ++++++++++------
 ...FullTextContainsWithoutOptionDescriptor.java |  68 +++++++++++++
 .../apache/hyracks/control/nc/io/IOManager.java |   8 +-
 .../hyracks/data/std/util/BinaryHashSet.java    |  20 ++--
 .../LSMInvertedIndexDataflowHelper.java         |   2 +-
 ...SMInvertedIndexSearchOperatorDescriptor.java |   6 +-
 ...InvertedIndexSearchOperatorNodePushable.java |  10 +-
 .../ondisk/OnDiskInvertedIndexFactory.java      |   2 +-
 .../search/AbstractTOccurrenceSearcher.java     |  30 ++++++
 .../search/DisjunctiveSearchModifier.java       |  53 ++++++++++
 .../DisjunctiveSearchModifierFactory.java       |  32 ++++++
 .../search/InvertedIndexSearchPredicate.java    |  15 ++-
 .../DelimitedUTF8StringBinaryTokenizer.java     |   6 ++
 .../tokenizers/IBinaryTokenizer.java            |   5 +
 .../NGramUTF8StringBinaryTokenizer.java         |   6 ++
 .../invertedindex/tokenizers/TokenizerInfo.java |  36 +++++++
 164 files changed, 4207 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
----------------------------------------------------------------------
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 8bb1b22..7d8dcd1 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
@@ -144,7 +144,8 @@ public class InvertedIndexPOperator extends IndexSearchPOperator {
                 metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
                 retainNull, jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
                 jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
-                jobGenParams.getSimilarityThreshold(), minFilterFieldIndexes, maxFilterFieldIndexes);
+                jobGenParams.getSimilarityThreshold(), minFilterFieldIndexes, maxFilterFieldIndexes,
+                jobGenParams.getIsFullTextSearch());
 
         // Contribute operator in hyracks job.
         builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
@@ -158,7 +159,8 @@ public class InvertedIndexPOperator extends IndexSearchPOperator {
             AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
             String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
             SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery)
+            throws AlgebricksException {
 
         try {
             IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
@@ -288,7 +290,7 @@ public class InvertedIndexPOperator extends IndexSearchPOperator {
                     invListsTypeTraits, invListsComparatorFactories, dataflowHelperFactory, queryTokenizerFactory,
                     searchModifierFactory, outputRecDesc, retainInput, retainMissing, context.getMissingWriterFactory(),
                     NoOpOperationCallbackFactory.INSTANCE, minFilterFieldIndexes, maxFilterFieldIndexes,
-                    LSMIndexUtil.getMetadataPageManagerFactory());
+                    LSMIndexUtil.getMetadataPageManagerFactory(), isFullTextSearchQuery);
             return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
         } catch (MetadataException e) {
             throw new AlgebricksException(e);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
----------------------------------------------------------------------
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/FullTextContainsParameterCheckRule.java
index e36b87d..e589065 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/FullTextContainsParameterCheckRule.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
+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.util.ConstantExpressionUtil;
@@ -34,6 +36,7 @@ import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -49,6 +52,13 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
     // parameter name and its value
     HashMap<MutableObject<ILogicalExpression>, MutableObject<ILogicalExpression>> paramValueMap;
 
+    // the last expression position before the option argument in the arguments array
+    private static final int LAST_EXPRESSION_POS_BEFORE_OPTION = 1;
+    // The number of anticipated arguments for a full-text query when a user doesn't provide any option.
+    private static final int FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS = 2;
+    // 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;
+
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
@@ -92,7 +102,19 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         FunctionIdentifier fi = funcExpr.getFunctionIdentifier();
 
+        // Collects the correct number of arguments - it can be 2 if a user doesn't provide any option.
+        int numberOfCorrectArguments = 0;
+        String functionName = "";
         if (fi == BuiltinFunctions.FULLTEXT_CONTAINS) {
+            numberOfCorrectArguments = FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS;
+            functionName = BuiltinFunctions.FULLTEXT_CONTAINS.getName();
+        } else if (fi == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+            numberOfCorrectArguments = FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS;
+            functionName = BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION.getName();
+        }
+
+        // If numberOfCorrectArguments is greater than zero, then this is a full-text search query.
+        if (numberOfCorrectArguments > 0) {
             // Don't need to check this operator again.
             context.addToDontApplySet(this, op);
 
@@ -100,21 +122,26 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
             List<Mutable<ILogicalExpression>> newExprs = new ArrayList<>();
 
             // The number of parameters should be three: exp1, exp2, and the option
-            if (oldExprs.size() != 3) {
+            if (oldExprs.size() != numberOfCorrectArguments) {
                 throw new AlgebricksException(
-                        BuiltinFunctions.FULLTEXT_CONTAINS.getName() + " should have three parameters.");
+                        functionName + " should have " + numberOfCorrectArguments + " parameters.");
             }
 
-            // The last expression is a record that contains the parameters. That's why we deduct -1.
-            for (int i = 0; i < oldExprs.size() - 1; i++) {
+            // The last expression before the option needs to be copied first.
+            for (int i = 0; i <= LAST_EXPRESSION_POS_BEFORE_OPTION; i++) {
                 newExprs.add(new MutableObject<ILogicalExpression>((ILogicalExpression) oldExprs.get(i).getValue()));
             }
 
             // Sanity check for the types of the first two parameters
-            checkFirstAndSecondParamter(oldExprs);
+            checkFirstAndSecondParamter(oldExprs, functionName);
 
             // Checks and transforms the actual full-text parameters.
-            checkAndSetDefaultValueForThirdParameter(oldExprs.get(2), newExprs);
+            if (numberOfCorrectArguments == FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS) {
+                checkValueForThirdParameter(oldExprs.get(2), newExprs);
+            } else {
+                // no option provided case: sets the default option here.
+                setDefaultValueForThirdParameter(newExprs);
+            }
 
             // Resets the last argument.
             funcExpr.getArguments().clear();
@@ -130,13 +157,13 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
      * Checks the correctness of the first and second argument. If the argument is a constant, we can check
      * it now. If the argument is not a constant, we will defer the checking until run-time.
      */
-    void checkFirstAndSecondParamter(List<Mutable<ILogicalExpression>> exprs) throws AlgebricksException {
+    void checkFirstAndSecondParamter(List<Mutable<ILogicalExpression>> exprs, String functionName)
+            throws AlgebricksException {
         // Check the first parameter - Expression1. If it's a constant, then we can check the type here.
         ILogicalExpression firstExpr = exprs.get(0).getValue();
         if (firstExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
                 && ConstantExpressionUtil.getConstantIaObjectType(firstExpr) != ATypeTag.STRING) {
-            throw new AlgebricksException("The first expression of "
-                    + BuiltinFunctions.FULLTEXT_CONTAINS.getName() + " should be a string.");
+            throw new AlgebricksException("The first expression of " + functionName + " should be a string.");
         }
 
         // Check the second parameter - Expression2. If it's a constant, then we can check the type here.
@@ -149,9 +176,8 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                 case ORDEREDLIST:
                     break;
                 default:
-                    throw new AlgebricksException(
-                            "The second expression of " + BuiltinFunctions.FULLTEXT_CONTAINS.getName()
-                                    + "should be a string, an unordered list, or an ordered list.");
+                    throw new AlgebricksException("The second expression of " + functionName
+                            + "should be a string, an unordered list, or an ordered list.");
             }
         }
     }
@@ -162,7 +188,7 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
      * @param expr
      * @throws AlgebricksException
      */
-    void checkAndSetDefaultValueForThirdParameter(Mutable<ILogicalExpression> expr,
+    void checkValueForThirdParameter(Mutable<ILogicalExpression> expr,
             List<Mutable<ILogicalExpression>> newArgs) throws AlgebricksException {
         // Get the last parameter - this should be a record-constructor.
         AbstractFunctionCallExpression openRecConsExpr = (AbstractFunctionCallExpression) expr.getValue();
@@ -243,4 +269,21 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                     + " or " + FullTextContainsDescriptor.DISJUNCTIVE_SEARCH_MODE_OPTION + ".");
         }
     }
+
+    /**
+     * Sets the default option value(s) when a user doesn't provide any option.
+     */
+    void setDefaultValueForThirdParameter(List<Mutable<ILogicalExpression>> newArgs)
+            throws AlgebricksException {
+        // 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)));
+
+        // Add this option as arguments to the ftcontains().
+        newArgs.add(new MutableObject<ILogicalExpression>(searchModeOptionExpr));
+        newArgs.add(new MutableObject<ILogicalExpression>(searchModeValExpr));
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
----------------------------------------------------------------------
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 0680183..e6cf12d 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
@@ -20,6 +20,7 @@ package org.apache.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -36,6 +37,8 @@ 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.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -296,9 +299,23 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     boolean jaccardSimilarity = optFuncExpr.getFuncExpr().getFunctionIdentifier().getName()
                             .startsWith("similarity-jaccard-check");
 
+                    // Full-text search consideration: an (un)ordered list of string type can be compatible with string
+                    // type. i.e. an (un)ordered list can be provided as arguments to a string type field index.
+                    List<IAType> elementTypes = matchedTypes;
+                    if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                            || optFuncExpr.getFuncExpr()
+                                    .getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+                        for (int j = 0; j < matchedTypes.size(); j++) {
+                            if (matchedTypes.get(j).getTypeTag() == ATypeTag.ORDEREDLIST
+                                    || matchedTypes.get(j).getTypeTag() == ATypeTag.UNORDEREDLIST) {
+                                elementTypes.set(j, ((AbstractCollectionType) matchedTypes.get(j)).getItemType());
+                            }
+                        }
+                    }
+
                     for (int j = 0; j < matchedTypes.size(); j++) {
                         for (int k = j + 1; k < matchedTypes.size(); k++) {
-                            typeMatch &= isMatched(matchedTypes.get(j), matchedTypes.get(k), jaccardSimilarity);
+                            typeMatch &= isMatched(elementTypes.get(j), elementTypes.get(k), jaccardSimilarity);
                         }
                     }
 
@@ -348,6 +365,10 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
     }
 
     private boolean isMatched(IAType type1, IAType type2, boolean useListDomain) throws AlgebricksException {
+        // Sanity check - two types can't be NULL in order to be matched.
+        if (type1 == null || type2 == null) {
+            return false;
+        }
         if (ATypeHierarchy.isSameTypeDomain(Index.getNonNullableType(type1).first.getTypeTag(),
                 Index.getNonNullableType(type2).first.getTypeTag(), useListDomain)) {
             return true;
@@ -546,7 +567,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
                     funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
                     datasetRecordVar, subTree.getMetaRecordType(), datasetMetaVar);
-            if (fieldName == null) {
+            if (fieldName.isEmpty()) {
                 return;
             }
         }
@@ -683,25 +704,25 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
             expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
-            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                //Otherwise the cast for childFuncExpr would fail
-                return null;
+            // Can't get a field name from a constant expression. So, return null.
+            if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                return Collections.emptyList();
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
         } else {
             UnnestOperator unnestOp = (UnnestOperator) op;
             expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return null;
+                return Collections.emptyList();
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
             if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
-                return null;
+                return Collections.emptyList();
             }
             expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
         }
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return null;
+            return Collections.emptyList();
         }
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
@@ -714,21 +735,21 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
             fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
             if (fieldName == null) {
-                return null;
+                return Collections.emptyList();
             }
             isFieldAccess = true;
             isByName = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
             Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
             if (idx == null) {
-                return null;
+                return Collections.emptyList();
             }
             fieldIndex = idx;
             isFieldAccess = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
             ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
             if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return null;
+                return Collections.emptyList();
             }
             ConstantExpression constExpr = (ConstantExpression) nameArg;
             AOrderedList orderedNestedFieldName = (AOrderedList) ((AsterixConstantValue) constExpr.getValue())
@@ -783,10 +804,10 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                         assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
                         parentFuncExpr, recordVar, metaType, metaVar);
 
-                if (parentFieldNames == null) {
+                if (parentFieldNames.isEmpty()) {
                     //Nested assign was not a field access.
                     //We will not use index
-                    return null;
+                    return Collections.emptyList();
                 }
 
                 if (!isByName) {
@@ -820,7 +841,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         }
 
         if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
-            return null;
+            return Collections.emptyList();
         }
         // We use a part of the field in edit distance computation
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
@@ -830,7 +851,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         // cannot apply an index.
         ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
         if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-            return null;
+            return Collections.emptyList();
         }
         LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
         // We look for the assign or unnest operator that produces curVar below
@@ -858,6 +879,6 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 }
             }
         }
-        return null;
+        return Collections.emptyList();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 661786c..2e6518b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -39,10 +39,13 @@ import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 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.hierachy.ATypeHierarchy;
 import org.apache.asterix.om.util.ConstantExpressionUtil;
@@ -77,6 +80,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOpe
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 
 /**
  * Static helper functions for rewriting plans using indexes.
@@ -139,7 +143,9 @@ public class AccessMethodUtils {
         }
         if (arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
             // The arguments of contains() function are asymmetrical, we can only use index if it is on the first argument
-            if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS) {
+            if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS
+                    || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                    || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
                 return false;
             }
             IAType expressionType = constantRuntimeResultType(arg1, context, typeEnvironment);
@@ -159,6 +165,15 @@ public class AccessMethodUtils {
             }
             constantExpressionType = expressionType;
             constExpression = arg2;
+
+            // For a full-text search query, if the given predicate is a constant and not a single keyword,
+            // i.e. it's a phrase, then we currently throw an exception since we don't support a phrase search
+            // yet in the full-text search.
+            if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                    && arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                checkFTSearchConstantExpression(constExpression);
+            }
+
             VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
             fieldVar = varExpr.getVariableReference();
         } else {
@@ -176,6 +191,62 @@ public class AccessMethodUtils {
         return true;
     }
 
+    /**
+     * Fetches each element and calls the check for the type and value in the given list using the given cursor.
+     */
+    private static void checkEachElementInFTSearchListPredicate(IACursor oListCursor)
+            throws AlgebricksException {
+        String argValue;
+        IAObject element;
+        while (oListCursor.next()) {
+            element = oListCursor.get();
+            if (element.getType() == BuiltinType.ASTRING) {
+                argValue = ConstantExpressionUtil.getStringConstant(element);
+                checkAndGenerateFTSearchExceptionForStringPhrase(argValue);
+            } else {
+                throw new AlgebricksException("Each element in the list should be a string in the Full-text search.");
+            }
+        }
+    }
+
+    // Checks whether a proper constant expression is in place for the full-text search.
+    // A proper constant expression in the full-text search should be among string, string type (Un)ordered list.
+    public static void checkFTSearchConstantExpression(ILogicalExpression constExpression) throws AlgebricksException {
+        IAObject objectFromExpr = ConstantExpressionUtil.getConstantIaObject(constExpression, null);
+        String arg2Value;
+        IACursor oListCursor;
+
+        switch (objectFromExpr.getType().getTypeTag()) {
+            case STRING:
+                arg2Value = ConstantExpressionUtil.getStringConstant(objectFromExpr);
+                checkAndGenerateFTSearchExceptionForStringPhrase(arg2Value);
+                break;
+            case ORDEREDLIST:
+                oListCursor = ConstantExpressionUtil.getOrderedListConstant(objectFromExpr).getCursor();
+                checkEachElementInFTSearchListPredicate(oListCursor);
+                break;
+            case UNORDEREDLIST:
+                oListCursor = ConstantExpressionUtil.getUnorderedListConstant(objectFromExpr).getCursor();
+                checkEachElementInFTSearchListPredicate(oListCursor);
+                break;
+            default:
+                throw new AlgebricksException(
+                        "A full-text Search predicate should be a string or an (un)ordered list.");
+        }
+    }
+
+    // Checks whether the given string is a phrase. If so, generates an exception since
+    // we don't support a phrase search in the full-text search yet.
+    public static void checkAndGenerateFTSearchExceptionForStringPhrase(String value) throws AlgebricksException {
+        for (int j = 0; j < value.length(); j++) {
+            if (DelimitedUTF8StringBinaryTokenizer.isSeparator(value.charAt(j))) {
+                throw new AlgebricksException(
+                        "Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted."
+                                + value.charAt(j));
+            }
+        }
+    }
+
     public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
             AccessMethodAnalysisContext analysisCtx) {
         LogicalVariable fieldVar1 = null;
@@ -209,15 +280,13 @@ public class AccessMethodUtils {
         if (!primaryKeysOnly) {
             switch (index.getIndexType()) {
                 case BTREE:
-                case SINGLE_PARTITION_WORD_INVIX:
-                case SINGLE_PARTITION_NGRAM_INVIX: {
                     dest.addAll(KeyFieldTypeUtils.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
-                }
-                case RTREE: {
+                case RTREE:
                     dest.addAll(KeyFieldTypeUtils.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
-                }
+                case SINGLE_PARTITION_WORD_INVIX:
+                case SINGLE_PARTITION_NGRAM_INVIX:
                 case LENGTH_PARTITIONED_NGRAM_INVIX:
                 case LENGTH_PARTITIONED_WORD_INVIX:
                 default:

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
----------------------------------------------------------------------
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 2dfe850..30866cf 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
@@ -36,6 +36,7 @@ import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.base.AFloat;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACollection;
 import org.apache.asterix.om.base.IAObject;
@@ -46,6 +47,8 @@ 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.util.ConstantExpressionUtil;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -81,6 +84,7 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchM
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveEditDistanceSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveListEditDistanceSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifierFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.search.DisjunctiveSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.EditDistanceSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ListEditDistanceSearchModifierFactory;
@@ -97,7 +101,8 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         JACCARD,
         EDIT_DISTANCE,
         CONJUNCTIVE_EDIT_DISTANCE,
-        INVALID
+        INVALID,
+        DISJUNCTIVE
     }
 
     private static List<FunctionIdentifier> funcIdents = new ArrayList<>();
@@ -107,6 +112,9 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         // For matching similarity-check functions. For example, similarity-jaccard-check returns a list of two items,
         // and the select condition will get the first list-item and check whether it evaluates to true.
         funcIdents.add(BuiltinFunctions.GET_ITEM);
+        // Full-text search function
+        funcIdents.add(BuiltinFunctions.FULLTEXT_CONTAINS);
+        funcIdents.add(BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION);
     }
 
     // These function identifiers are matched in this AM's analyzeFuncExprArgs(),
@@ -131,7 +139,9 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
 
-        if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS) {
+        if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS
+                || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                || funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
             boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx, context,
                     typeEnvironment);
             if (!matches) {
@@ -828,6 +838,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS) {
             jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
             jobGenParams.setSimilarityThreshold(new AsterixConstantValue(AMissing.MISSING));
+            return;
         }
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
             jobGenParams.setSearchModifierType(SearchModifierType.JACCARD);
@@ -835,6 +846,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             jobGenParams.setSimilarityThreshold(
                     ((ConstantExpression) optFuncExpr.getConstantExpr(optFuncExpr.getNumConstantExpr() - 1))
                             .getValue());
+            return;
         }
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK
                 || optFuncExpr.getFuncExpr()
@@ -848,9 +860,42 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             jobGenParams.setSimilarityThreshold(
                     ((ConstantExpression) optFuncExpr.getConstantExpr(optFuncExpr.getNumConstantExpr() - 1))
                             .getValue());
+            return;
+        }
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                || optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+            // Let the job Gen pass the full-text search information.
+            jobGenParams.setIsFullTextSearch(true);
+
+            // 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.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.DISJUNCTIVE;
+        }
+        // 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 {
@@ -883,6 +928,11 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             return isContainsFuncOptimizable(index, optFuncExpr);
         }
 
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
+                || optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+            return isFullTextContainsFuncOptimizable(index, optFuncExpr);
+        }
+
         return false;
     }
 
@@ -1012,6 +1062,40 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         return false;
     }
 
+    private boolean isFullTextContainsFuncCompatible(ATypeTag typeTag, IndexType indexType) {
+        //We can only optimize contains with full-text indexes.
+        return (typeTag == ATypeTag.STRING || typeTag == ATypeTag.ORDEREDLIST || typeTag == ATypeTag.UNORDEREDLIST)
+                && indexType == IndexType.SINGLE_PARTITION_WORD_INVIX;
+    }
+
+    // Does full-text search can utilize the given index?
+    private boolean isFullTextContainsFuncOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        if (optFuncExpr.getNumLogicalVars() == 2) {
+            return isFullTextContainsFuncJoinOptimizable(index, optFuncExpr);
+        } else {
+            return isFullTextContainsFuncSelectOptimizable(index, optFuncExpr);
+        }
+    }
+
+    // Checks whether the given index is compatible with full-text search and
+    // the type of the constant search predicate is STRING, ORDEREDLIST, or UNORDEREDLIST
+    private boolean isFullTextContainsFuncSelectOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        AsterixConstantValue strConstVal =
+                (AsterixConstantValue) ((ConstantExpression) optFuncExpr.getConstantExpr(0)).getValue();
+        IAObject strObj = strConstVal.getObject();
+        ATypeTag typeTag = strObj.getType().getTypeTag();
+
+        return isFullTextContainsFuncCompatible(typeTag, index.getIndexType());
+    }
+
+    private boolean isFullTextContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        if (index.isEnforcingKeyFileds()) {
+            return isFullTextContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+        } else {
+            return isFullTextContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
+        }
+    }
+
     private ScalarFunctionCallExpression findTokensFunc(FunctionIdentifier funcId, IOptimizableFuncExpr optFuncExpr,
             int subTreeIndex) {
         //find either a gram-tokens or a word-tokens function that exists in optFuncExpr.subTrees' assignsAndUnnests
@@ -1150,15 +1234,15 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
     public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
             IAObject simThresh, Index index) throws AlgebricksException {
         switch (searchModifierType) {
-            case CONJUNCTIVE: {
+            case CONJUNCTIVE:
                 return new ConjunctiveSearchModifierFactory();
-            }
-            case JACCARD: {
+            case DISJUNCTIVE:
+                return new DisjunctiveSearchModifierFactory();
+            case JACCARD:
                 float jaccThresh = ((AFloat) simThresh).getFloatValue();
                 return new JaccardSearchModifierFactory(jaccThresh);
-            }
             case EDIT_DISTANCE:
-            case CONJUNCTIVE_EDIT_DISTANCE: {
+            case CONJUNCTIVE_EDIT_DISTANCE:
                 int edThresh = 0;
                 try {
                     edThresh = ((AInt32) ATypeHierarchy.convertNumericTypeObject(simThresh, ATypeTag.INT32))
@@ -1191,10 +1275,8 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                                 + "' for index type '" + index.getIndexType() + "'");
                     }
                 }
-            }
-            default: {
+            default:
                 throw new AlgebricksException("Unknown search modifier type '" + searchModifierType + "'.");
-            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index b6786ad..05122cd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -42,6 +42,16 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
     protected ATypeTag searchKeyType;
     protected List<LogicalVariable> keyVarList;
     protected List<LogicalVariable> nonKeyVarList;
+    // TODO: Currently, we don't have positional information in an inverted index.
+    // Thus, we can't support the phrase search yet. So, for the full-text search,
+    // if a query predicate contains a phrase, we need to generate an exception.
+    // The following variable serves this purpose. i.e. Checks whether the query is a full-text search query or not.
+    protected boolean isFullTextSearchQuery = false;
+    protected static final int SEARCH_MODIFIER_INDEX = 0;
+    protected static final int SIM_THRESHOLD_INDEX = 1;
+    protected static final int SEARCH_KEY_TYPE_INDEX = 2;
+    protected static final int IS_FULLTEXT_SEARCH_INDEX = 3;
+    protected static final int KEY_VAR_INDEX = 4;
 
     public InvertedIndexJobGenParams() {
     }
@@ -55,6 +65,10 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
         this.searchModifierType = searchModifierType;
     }
 
+    public void setIsFullTextSearch(boolean isFullTextSearchQuery) {
+        this.isFullTextSearchQuery = isFullTextSearchQuery;
+    }
+
     public void setSimilarityThreshold(IAlgebricksConstantValue similarityThreshold) {
         this.similarityThreshold = similarityThreshold;
     }
@@ -78,6 +92,9 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
         // Write search key type.
         funcArgs.add(
                 new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType.ordinal())));
+        // Write full-text search information.
+        funcArgs.add(
+                new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(isFullTextSearchQuery)));
         // Write key var list.
         writeVarList(keyVarList, funcArgs);
         // Write non-key var list.
@@ -91,16 +108,18 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
         // Read search modifier type.
-        int searchModifierOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
+        int searchModifierOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + SEARCH_MODIFIER_INDEX));
         searchModifierType = SearchModifierType.values()[searchModifierOrdinal];
         // Read similarity threshold. Concrete type depends on search modifier.
-        similarityThreshold = (((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue());
+        similarityThreshold = ((ConstantExpression) funcArgs.get(index + SIM_THRESHOLD_INDEX).getValue()).getValue();
         // Read type of search key.
-        int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + 2));
+        int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + SEARCH_KEY_TYPE_INDEX));
         searchKeyType = ATypeTag.values()[typeTagOrdinal];
+        // Read full-text search information.
+        isFullTextSearchQuery = AccessMethodUtils.getBooleanConstant(funcArgs.get(index + IS_FULLTEXT_SEARCH_INDEX));
         // Read key var list.
         keyVarList = new ArrayList<LogicalVariable>();
-        readVarList(funcArgs, index + 3, keyVarList);
+        readVarList(funcArgs, index + KEY_VAR_INDEX, keyVarList);
         // TODO: We could possibly simplify things if we did read the non-key var list here.
         // We don't need to read the non-key var list.
         nonKeyVarList = null;
@@ -110,6 +129,10 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
         return searchModifierType;
     }
 
+    public boolean getIsFullTextSearch() {
+        return isFullTextSearchQuery;
+    }
+
     public IAlgebricksConstantValue getSimilarityThreshold() {
         return similarityThreshold;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
index 144531e..dc9ff17 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
@@ -153,6 +153,7 @@ import org.apache.asterix.runtime.evaluators.functions.EditDistanceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.EditDistanceListIsFilterableDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.EditDistanceStringIsFilterableDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GramTokensDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.HashedGramTokensDescriptor;
@@ -598,6 +599,7 @@ public class FunctionCollection {
 
         // full-text function
         functionsToInjectUnkownHandling.add(FullTextContainsDescriptor.FACTORY);
+        functionsToInjectUnkownHandling.add(FullTextContainsWithoutOptionDescriptor.FACTORY);
 
         // Record functions.
         functionsToInjectUnkownHandling.add(GetRecordFieldsDescriptor.FACTORY);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains-without-option.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains-without-option.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains-without-option.aql
new file mode 100644
index 0000000..eee411c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains-without-option.aql
@@ -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.
+ */
+
+/*
+ * Description    : Tests whether an full-text index is applied to optimize a selection query
+ *                : using the ftcontains function. The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index fulltext_index_title on MyData(title) type fulltext;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_fulltext-contains-without-option.adm";
+
+for $o in dataset MyData
+where ftcontains($o.title, "database")
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains.aql
new file mode 100644
index 0000000..bc697df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/fulltext-contains.aql
@@ -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.
+ */
+
+/*
+ * Description    : Tests whether an full-text index is applied to optimize a selection query
+ *                : using the ftcontains function. The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index fulltext_index_title on MyData(title) type fulltext;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_fulltext-contains.adm";
+
+for $o in dataset MyData
+where ftcontains($o.title, "database", {"mode":"any"})
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains-without-option.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains-without-option.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains-without-option.plan
new file mode 100644
index 0000000..8e1f97c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains-without-option.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains.plan
new file mode 100644
index 0000000..fc894d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/fulltext-contains.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.1.ddl.aql
index 6c731a9..3619c65 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.1.ddl.aql
@@ -23,6 +23,9 @@
  *                  : query #3 - single string value query
  *                  : query #4 - single string value in an ordered list query
  *                  : query #5 - single string value in an unordered list query
+ *                  : query #6 - the same as #3, but without any option
+ *                  : query #7 - the same as #4, but without any option
+ *                  : query #8 - the same as #5, but without any option
  *  Expected Result : Success
  *
 */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.6.query.aql
new file mode 100644
index 0000000..93e7d2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.6.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, "database")
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.7.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.7.query.aql
new file mode 100644
index 0000000..7409e15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.7.query.aql
@@ -0,0 +1,26 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, ["database"])
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.8.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.8.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.8.query.aql
new file mode 100644
index 0000000..8ef67dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-01/fulltext-01.8.query.aql
@@ -0,0 +1,26 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, {{"database"}})
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.1.ddl.aql
index eba30c4..e374593 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.1.ddl.aql
@@ -28,6 +28,8 @@
  *                  :            in this case, "any" option that enforces a disjunctive search will be applied.
  *                  : query #6 - the same as query #6, but with a different option - "all"
  *                  :            in this case, we explicitly specify "all" option that enforces a conjunctive search.
+ *                  : query #7 - the same as query #4, but without any option that is equivalent to "all".
+ *                  : query #8 - the same as query #6, but without any option that is equivalent to "all".
  *  Expected Result : Success
  *
 */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.7.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.7.query.aql
new file mode 100644
index 0000000..16478b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.7.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, ["object","database"])
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.8.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.8.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.8.query.aql
new file mode 100644
index 0000000..cd8165c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-02/fulltext-02.8.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, {{"object","database"}})
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.1.ddl.aql
index 9096d89..8b35f2c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.1.ddl.aql
@@ -33,6 +33,9 @@
  *                  :            in this case, "any" option that enforces a disjunctive search will be applied.
  *                  : query #8 - the same as query #7, but with a different option - "all"
  *                  :            in this case, we explicitly specify "all" option that enforces a conjunctive search.
+ *                  : query #9 - the same as query #4, but without any option that is equivalent to "all".
+ *                  : query #10 - the same as query #6, but without any option that is equivalent to "all".
+ *                  : query #11 - the same as query #8, but without any option that is equivalent to "all".
  *  Expected Result : Success
  *
 */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.10.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.10.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.10.query.aql
new file mode 100644
index 0000000..402c25c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.10.query.aql
@@ -0,0 +1,26 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+let $list := {{"object", "database"}}
+where ftcontains($o.title, $list)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.11.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.11.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.11.query.aql
new file mode 100644
index 0000000..06adaaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.11.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+where ftcontains($o.title, for $list in dataset MyKeywordData return $list.keyword_text)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.9.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.9.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.9.query.aql
new file mode 100644
index 0000000..93df951
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-03/fulltext-03.9.query.aql
@@ -0,0 +1,26 @@
+/*
+ * 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 dataverse test;
+
+for $o in dataset MyData
+let $list := ["object", "database"]
+where ftcontains($o.title, $list)
+order by $o.id
+return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-05/fulltext-05.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-05/fulltext-05.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-05/fulltext-05.1.ddl.aql
index 011a86e..a80fbed 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-05/fulltext-05.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-05/fulltext-05.1.ddl.aql
@@ -54,5 +54,3 @@ create dataset MyData(MyRecord)
 
 create dataset MyKeywordData(MyKeyword)
   primary key keyword_text;
-
-create index fulltext_index_title on MyData(title) type fulltext;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-06/fulltext-06.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-06/fulltext-06.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-06/fulltext-06.1.ddl.aql
index e155ba8..bfed240 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-06/fulltext-06.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-06/fulltext-06.1.ddl.aql
@@ -54,5 +54,3 @@ create dataset MyData(MyRecord)
 
 create dataset MyKeywordData(MyKeyword)
   primary key keyword_text;
-
-create index fulltext_index_title on MyData(title) type fulltext;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.1.ddl.aql
index 13e5506..7cdc675 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.1.ddl.aql
@@ -20,11 +20,14 @@
 /*
  *  Description     : Full-text search non-index test
  *                  : This test is intended to verify that the full-text search works as expected.
+ *                  : But, the form of the query is join. So, each keyword from the outer dataset will be processed
+ *                  : separately. Thus, query #3 and query #4 should generate the same result.
  *                  : query #3 - two string values in [an ordered list] query with "any" option.
  *                  :            an ordered list is first initialized by let clause and is being used.
  *                  :            in this case, "any" option that enforces a disjunctive search will be applied.
  *                  : query #4 - the same as query #3, but with a different option - "all"
  *                  :            in this case, we explicitly specify "all" option that enforces a conjunctive search.
+ *                  : query #5 - the same as query #4, but without any option that is equivalent to "all".
  *  Expected Result : Success
  *
 */
@@ -57,5 +60,3 @@ create dataset MyData(MyRecord)
 
 create dataset MyKeywordData(MyKeyword)
   primary key keyword_text;
-
-create index fulltext_index_title on MyData(title) type fulltext;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c49405aa/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.5.query.aql
new file mode 100644
index 0000000..ceeb2ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fulltext/fulltext-07/fulltext-07.5.query.aql
@@ -0,0 +1,26 @@
+/*
+ * 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 dataverse test;
+
+for $k in dataset MyKeywordData
+for $t in dataset MyData
+where ftcontains($t.title, $k.keyword_text, {"mode":"all"})
+order by $t.id
+return {"id":$t.id}