You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by sj...@apache.org on 2016/08/08 19:11:27 UTC

vxquery git commit: VXQUERY -218 Add delete-index function and XTests, Junit tests for MetaFileUtils.

Repository: vxquery
Updated Branches:
  refs/heads/master c21702bc3 -> 446a1e565


VXQUERY -218 Add delete-index function and XTests, Junit tests for MetaFileUtils.

This fix contains following changes.
1. Adding delete index function.
2. Modifying return types of create,
 update and delete to boolean.
3. Updated the metadata file structure.
4. Added XTest for delete index and changed the
 result files of create and update index.
5. JUnit test cases for MetafileUtils.


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

Branch: refs/heads/master
Commit: 446a1e565a36b555aa97b8fe8efa653e4b7331ad
Parents: c21702b
Author: Steven Glenn Jacobs <sj...@ucr.edu>
Authored: Mon Aug 8 12:09:45 2016 -0700
Committer: Steven Glenn Jacobs <sj...@ucr.edu>
Committed: Mon Aug 8 12:09:45 2016 -0700

----------------------------------------------------------------------
 .../vxquery/functions/builtin-functions.xml     |  26 +--
 .../functions/index/CaseSensitiveAnalyzer.java  |   6 +-
 .../index/CaseSensitiveQueryParser.java         |   4 +-
 ...ctionFromIndexUnnestingEvaluatorFactory.java |  16 +-
 .../IndexConstructorScalarEvaluatorFactory.java |  17 +-
 .../functions/index/IndexConstructorUtil.java   |  43 +++--
 .../index/IndexDeleteEvaluatorFactory.java      |  83 ++++++++++
 .../index/IndexUpdaterEvaluatorFactory.java     |  16 +-
 .../functions/index/updateIndex/Constants.java  |   2 +-
 .../index/updateIndex/IndexUpdater.java         | 134 ++++++++++++----
 .../index/updateIndex/MetaFileUtil.java         | 150 ++++++++++++++---
 .../index/updateIndex/VXQueryIndex.java         |  42 +++++
 .../index/updateIndex/XmlMetadata.java          |  25 ++-
 .../updateIndex/XmlMetadataCollection.java      |  62 ++++++++
 .../vxquery/indexing/MetaFileUtilTest.java      | 159 +++++++++++++++++++
 .../apache/vxquery/indexing/TestConstants.java  | 113 +++++++++++++
 .../src/test/resources/collection/catalog.xml   |  43 +++++
 .../resources/collection/catalog_edited.xml     |  43 +++++
 .../Indexing/createIndex.txt                    |   1 +
 .../Indexing/deleteIndex.txt                    |   1 +
 .../Indexing/updateIndex.txt                    |   1 +
 .../Queries/XQuery/Indexing/deleteIndex.xq      |  19 +++
 .../src/test/resources/cat/IndexingQueries.xml  |   5 +
 23 files changed, 893 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
index e1b6a7a..1cbda7a 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
+++ b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
@@ -132,18 +132,26 @@
     <function name="fn:build-index-on-collection">
         <param name="collection-folder" type="xs:string?"/>
         <param name="index-folder" type="xs:string?"/>
-        <return type="node()*"/>
+        <return type="xs:boolean"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.index.IndexConstructorScalarEvaluatorFactory"/>
     </function>
 
     <!-- fn:update-index($indexFolder  as xs:string?) as  node()* -->
     <function name="fn:update-index">
         <param name="index-folder" type="xs:string?"/>
-        <return type="node()*"/>
+        <return type="xs:boolean"/>
         <runtime type="scalar"
                  class="org.apache.vxquery.runtime.functions.index.IndexUpdaterEvaluatorFactory"/>
     </function>
 
+    <!-- fn:delete-index($indexFolder  as xs:string?) as  node()* -->
+    <function name="fn:delete-index">
+        <param name="index-folder" type="xs:string?"/>
+        <return type="xs:boolean"/>
+        <runtime type="scalar"
+                 class="org.apache.vxquery.runtime.functions.index.IndexDeleteEvaluatorFactory"/>
+    </function>
+
     <!-- fn:collection-from-index($indexfolder  as xs:string?, $elementpath as xs:string?) as  node()* -->
     <function name="fn:collection-from-index">
         <param name="index-folder" type="xs:string?"/>
@@ -323,21 +331,21 @@
         <return type="document-node()?"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.node.FnDocScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- jn:json-doc($uri as  xs:string?) as json-item()?  -->
     <function name="jn:json-doc">
         <param name="uri" type="xs:string?"/>
         <return type="json-item()?"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnDocScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- jn:parse-json($arg as  xs:string?) as json-item()*  -->
     <function name="jn:parse-json">
         <param name="arg" type="xs:string?"/>
         <return type="json-item()*"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnParseJsonScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- jn:parse-json($arg as  xs:string?, $options as object()) as json-item()*  -->
     <function name="jn:parse-json">
         <param name="arg" type="xs:string?"/>
@@ -1171,7 +1179,7 @@
             <argument value="0"/>
         </property>
     </function>
-    
+
     <!-- fn:serialize($data  as item()*) as  xs:string  -->
     <function name="fn:serialize">
         <param name="data" type="item()*"/>
@@ -1185,7 +1193,7 @@
         <return type="xs:string*"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnKeysScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- jn:members($arg as item()*) as item()* -->
     <function name="jn:members">
         <param name="arg" type="item()*"/>
@@ -1198,14 +1206,14 @@
         <return type="js:null"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnNullScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- libjn:descendant-arrays($sequence as item()*) as array()* -->
     <function name="libjn:descendant-arrays">
         <param name="sequence" type="item()*"/>
         <return type="json-item()*"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.LibjnDescendantArraysScalarEvaluatorFactory"/>
     </function>
-    
+
     <!-- libjn:flatten($sequence as item()*) as item()* -->
     <function name="libjn:flatten">
         <param name="sequence" type="item()*"/>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveAnalyzer.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveAnalyzer.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveAnalyzer.java
index 803aeee..f832362 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveAnalyzer.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveAnalyzer.java
@@ -17,9 +17,6 @@
 
 package org.apache.vxquery.runtime.functions.index;
 
-import java.io.IOException;
-import java.io.Reader;
-
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.core.StopAnalyzer;
@@ -34,6 +31,9 @@ import org.apache.lucene.analysis.util.StopwordAnalyzerBase;
 import org.apache.lucene.analysis.util.WordlistLoader;
 import org.apache.lucene.util.Version;
 
+import java.io.IOException;
+import java.io.Reader;
+
 /**
  * Filters {@link StandardTokenizer} with {@link StandardFilter},
  * and {@link StopFilter}, using a list of

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveQueryParser.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveQueryParser.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveQueryParser.java
index 7cb0a18..864f514 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveQueryParser.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CaseSensitiveQueryParser.java
@@ -16,8 +16,6 @@
 */
 package org.apache.vxquery.runtime.functions.index;
 
-import java.io.StringReader;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.classic.FastCharStream;
@@ -25,6 +23,8 @@ import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.Query;
 
+import java.io.StringReader;
+
 public class CaseSensitiveQueryParser extends QueryParser {
 
     public CaseSensitiveQueryParser(String f, Analyzer a) {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CollectionFromIndexUnnestingEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CollectionFromIndexUnnestingEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CollectionFromIndexUnnestingEvaluatorFactory.java
index 9bd6b92..ea78de5 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CollectionFromIndexUnnestingEvaluatorFactory.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/CollectionFromIndexUnnestingEvaluatorFactory.java
@@ -16,14 +16,6 @@
  */
 package org.apache.vxquery.runtime.functions.index;
 
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,6 +49,14 @@ import org.apache.vxquery.xmlparser.TreeNodeIdProvider;
 import org.xml.sax.Attributes;
 import org.xml.sax.SAXException;
 
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 public class CollectionFromIndexUnnestingEvaluatorFactory extends AbstractTaggedValueArgumentUnnestingEvaluatorFactory {
     private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorScalarEvaluatorFactory.java
index c3776d9..cdae2cf 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorScalarEvaluatorFactory.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorScalarEvaluatorFactory.java
@@ -16,8 +16,6 @@
 */
 package org.apache.vxquery.runtime.functions.index;
 
-import java.io.DataInputStream;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -28,12 +26,17 @@ import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
 import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
+import org.apache.vxquery.datamodel.values.XDMConstants;
+import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
 import org.apache.vxquery.xmlparser.ITreeNodeIdProvider;
 import org.apache.vxquery.xmlparser.TreeNodeIdProvider;
 
+import javax.xml.bind.JAXBException;
+import java.io.DataInputStream;
+
 public class IndexConstructorScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
     //Creates one Lucene doc per file
 
@@ -61,8 +64,14 @@ public class IndexConstructorScalarEvaluatorFactory extends AbstractTaggedValueA
 
             @Override
             protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
-                IndexConstructorUtil.evaluate(args, result, stringp, bbis, di, sb, abvs, nodeIdProvider, abvsFileNode,
-                        nodep, false, nodeId);
+                try {
+                    IndexConstructorUtil
+                            .evaluate(args, result, stringp, bbis, di, sb, abvs, nodeIdProvider, abvsFileNode,
+                            nodep, false, nodeId);
+                    XDMConstants.setTrue(result);
+                } catch (JAXBException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
+                }
             }
 
         };

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorUtil.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorUtil.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorUtil.java
index 0d1dbf8..126ddf5 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorUtil.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexConstructorUtil.java
@@ -32,7 +32,6 @@ import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 import org.apache.vxquery.index.IndexDocumentBuilder;
-import org.apache.vxquery.runtime.functions.index.updateIndex.Constants;
 import org.apache.vxquery.runtime.functions.index.updateIndex.MetaFileUtil;
 import org.apache.vxquery.runtime.functions.index.updateIndex.XmlMetadata;
 import org.apache.vxquery.runtime.functions.util.FunctionHelper;
@@ -40,12 +39,14 @@ import org.apache.vxquery.xmlparser.IParser;
 import org.apache.vxquery.xmlparser.ITreeNodeIdProvider;
 import org.apache.vxquery.xmlparser.XMLParser;
 
+import javax.xml.bind.JAXBException;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Paths;
 import java.security.NoSuchAlgorithmException;
+import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -55,10 +56,9 @@ public class IndexConstructorUtil {
     static ConcurrentHashMap<String, XmlMetadata> metadataMap = new ConcurrentHashMap<>();
 
     public static void evaluate(TaggedValuePointable[] args, IPointable result, UTF8StringPointable stringp,
-                                ByteBufferInputStream bbis, DataInputStream di, SequenceBuilder sb,
-                                ArrayBackedValueStorage abvs, ITreeNodeIdProvider nodeIdProvider,
-                                ArrayBackedValueStorage abvsFileNode, TaggedValuePointable nodep,
-                                boolean isElementPath, String nodeId) throws SystemException {
+            ByteBufferInputStream bbis, DataInputStream di, SequenceBuilder sb, ArrayBackedValueStorage abvs,
+            ITreeNodeIdProvider nodeIdProvider, ArrayBackedValueStorage abvsFileNode, TaggedValuePointable nodep,
+            boolean isElementPath, String nodeId) throws SystemException, JAXBException {
         String collectionFolder;
         String indexFolder;
         TaggedValuePointable collectionTVP = args[0];
@@ -83,6 +83,7 @@ public class IndexConstructorUtil {
 
             metaFileUtil = MetaFileUtil.create(indexFolder);
             isMetaFilePresent = metaFileUtil.isMetaFilePresent();
+            metaFileUtil.setCollectionForIndex(indexFolder, collectionFolder);
 
         } catch (IOException e) {
             throw new SystemException(ErrorCode.SYSE0001, e);
@@ -111,13 +112,9 @@ public class IndexConstructorUtil {
                     nodeId);
 
             if (!isMetaFilePresent) {
-                // Add collection information to the map.
-                XmlMetadata data = new XmlMetadata();
-                data.setPath(collectionFolder);
-                metadataMap.put(Constants.COLLECTION_ENTRY, data);
-
                 // Write metadata map to a file.
-                metaFileUtil.writeMetaFile(metadataMap);
+                metaFileUtil.updateMetadataMap(metadataMap, indexFolder);
+                metaFileUtil.writeMetadataToFile();
             }
 
             //This makes write slower but search faster.
@@ -136,26 +133,26 @@ public class IndexConstructorUtil {
      * it indexes that document node.
      */
     public static void indexXmlFiles(File collectionDirectory, IndexWriter writer, boolean isElementPath,
-                                     TaggedValuePointable nodep, ArrayBackedValueStorage abvsFileNode,
-                                     ITreeNodeIdProvider nodeIdProvider, SequenceBuilder sb,
-                                     ByteBufferInputStream bbis, DataInputStream di, String nodeId)
+            TaggedValuePointable nodep, ArrayBackedValueStorage abvsFileNode, ITreeNodeIdProvider nodeIdProvider,
+            SequenceBuilder sb, ByteBufferInputStream bbis, DataInputStream di, String nodeId)
             throws SystemException, IOException {
 
+        SimpleDateFormat sdf = new SimpleDateFormat("dd/MM/yyyy, HH:mm:ss");
 
         for (File file : collectionDirectory.listFiles()) {
 
             if (readableXmlFile(file.getPath())) {
                 abvsFileNode.reset();
 
-                IndexDocumentBuilder ibuilder = getIndexBuilder(file, writer, nodep, abvsFileNode, nodeIdProvider,
-                        bbis, di, nodeId);
+                IndexDocumentBuilder ibuilder = getIndexBuilder(file, writer, nodep, abvsFileNode, nodeIdProvider, bbis,
+                        di, nodeId);
 
                 ibuilder.printStart();
-
                 if (!isMetaFilePresent) {
                     XmlMetadata xmlMetadata = new XmlMetadata();
                     xmlMetadata.setPath(file.getCanonicalPath());
                     xmlMetadata.setFileName(file.getName());
+                    xmlMetadata.setLastModified(sdf.format(file.lastModified()));
                     try {
                         xmlMetadata.setMd5(metaFileUtil.generateMD5(file));
                     } catch (NoSuchAlgorithmException e) {
@@ -175,15 +172,12 @@ public class IndexConstructorUtil {
         return (path.toLowerCase().endsWith(".xml") || path.toLowerCase().endsWith(".xml.gz"));
     }
 
-
     /**
      * Separated from create index method so that it could be used as a helper function in IndexUpdater
      */
-    public static IndexDocumentBuilder getIndexBuilder(File file, IndexWriter writer,
-                                                       TaggedValuePointable nodep, ArrayBackedValueStorage abvsFileNode,
-                                                       ITreeNodeIdProvider nodeIdProvider,
-                                                       ByteBufferInputStream bbis, DataInputStream di, String nodeId)
-            throws IOException {
+    public static IndexDocumentBuilder getIndexBuilder(File file, IndexWriter writer, TaggedValuePointable nodep,
+            ArrayBackedValueStorage abvsFileNode, ITreeNodeIdProvider nodeIdProvider, ByteBufferInputStream bbis,
+            DataInputStream di, String nodeId) throws IOException {
 
         //Get the document node
         IParser parser = new XMLParser(false, nodeIdProvider, nodeId);
@@ -195,5 +189,4 @@ public class IndexConstructorUtil {
         //Creates one lucene doc per file
         return new IndexDocumentBuilder(nodep, writer, file.getCanonicalPath());
     }
-
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexDeleteEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexDeleteEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexDeleteEvaluatorFactory.java
new file mode 100644
index 0000000..e713b20
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexDeleteEvaluatorFactory.java
@@ -0,0 +1,83 @@
+/*
+* 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.vxquery.runtime.functions.index;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
+import org.apache.vxquery.datamodel.values.XDMConstants;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+import org.apache.vxquery.runtime.functions.index.updateIndex.IndexUpdater;
+import org.apache.vxquery.xmlparser.ITreeNodeIdProvider;
+import org.apache.vxquery.xmlparser.TreeNodeIdProvider;
+
+import javax.xml.bind.JAXBException;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Delete the index of a given index directory
+ */
+public class IndexDeleteEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+    public IndexDeleteEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+        final UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        final TaggedValuePointable nodep = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+        final ByteBufferInputStream bbis = new ByteBufferInputStream();
+        final DataInputStream di = new DataInputStream(bbis);
+        final SequenceBuilder sb = new SequenceBuilder();
+        final ArrayBackedValueStorage abvsFileNode = new ArrayBackedValueStorage();
+        final int partition = ctx.getTaskAttemptId().getTaskId().getPartition();
+        final String nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
+        final ITreeNodeIdProvider nodeIdProvider = new TreeNodeIdProvider((short) partition);
+
+        return new AbstractTaggedValueArgumentScalarEvaluator(args) {
+
+            @Override
+            protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+                IndexUpdater updater = new IndexUpdater(args, result, stringp, bbis, di, sb, abvs, nodeIdProvider,
+                        abvsFileNode, nodep, nodeId);
+                try {
+                    updater.setup();
+                    updater.deleteAllIndexes();
+                    XDMConstants.setTrue(result);
+                } catch (IOException | NoSuchAlgorithmException | JAXBException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
+                }
+
+            }
+
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexUpdaterEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexUpdaterEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexUpdaterEvaluatorFactory.java
index 0231f3d..3fbc279 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexUpdaterEvaluatorFactory.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/IndexUpdaterEvaluatorFactory.java
@@ -26,6 +26,8 @@ import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
 import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
+import org.apache.vxquery.datamodel.values.XDMConstants;
+import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
@@ -33,6 +35,7 @@ import org.apache.vxquery.runtime.functions.index.updateIndex.IndexUpdater;
 import org.apache.vxquery.xmlparser.ITreeNodeIdProvider;
 import org.apache.vxquery.xmlparser.TreeNodeIdProvider;
 
+import javax.xml.bind.JAXBException;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.security.NoSuchAlgorithmException;
@@ -46,7 +49,8 @@ public class IndexUpdaterEvaluatorFactory extends AbstractTaggedValueArgumentSca
     }
 
     @Override
-    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) throws AlgebricksException {
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
         final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
         final UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
         final TaggedValuePointable nodep = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
@@ -65,9 +69,13 @@ public class IndexUpdaterEvaluatorFactory extends AbstractTaggedValueArgumentSca
                 IndexUpdater updater = new IndexUpdater(args, result, stringp, bbis, di, sb, abvs, nodeIdProvider,
                         abvsFileNode, nodep, nodeId);
                 try {
-                    updater.evaluate();
-                } catch (IOException | NoSuchAlgorithmException e) {
-                    e.printStackTrace();
+                    updater.setup();
+                    updater.updateIndex();
+                    updater.updateMetadataFile();
+                    updater.exit();
+                    XDMConstants.setTrue(result);
+                } catch (IOException | NoSuchAlgorithmException | JAXBException e) {
+                    throw new SystemException(ErrorCode.SYSE0001, e);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/Constants.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/Constants.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/Constants.java
index 321d348..9aebfb4 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/Constants.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/Constants.java
@@ -21,6 +21,6 @@ package org.apache.vxquery.runtime.functions.index.updateIndex;
  */
 public class Constants {
     public static String FIELD_PATH = "path";
-    public static String META_FILE_NAME = "metaFile.file";
+    public static String META_FILE_NAME = "vxquery_index.xml";
     public static String COLLECTION_ENTRY = "collection";
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/IndexUpdater.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/IndexUpdater.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/IndexUpdater.java
index 11621a7..4588282 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/IndexUpdater.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/IndexUpdater.java
@@ -37,12 +37,15 @@ import org.apache.vxquery.runtime.functions.index.CaseSensitiveAnalyzer;
 import org.apache.vxquery.runtime.functions.index.IndexConstructorUtil;
 import org.apache.vxquery.xmlparser.ITreeNodeIdProvider;
 
+import javax.xml.bind.JAXBException;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.NoSuchAlgorithmException;
+import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
@@ -67,13 +70,17 @@ public class IndexUpdater {
     private String nodeId;
     private IndexWriter indexWriter;
     private Set<String> pathsFromFileList;
+    private String collectionFolder;
+    private XmlMetadata collectionMetadata;
+    private String indexFolder;
     private Logger LOGGER = Logger.getLogger("Index Updater");
+    private SimpleDateFormat sdf = new SimpleDateFormat("dd/MM/yyyy HH:mm:ss");
 
     //TODO : Implement for paralleizing
     public IndexUpdater(TaggedValuePointable[] args, IPointable result, UTF8StringPointable stringp,
-                        ByteBufferInputStream bbis, DataInputStream di, SequenceBuilder sb, ArrayBackedValueStorage abvs,
-                        ITreeNodeIdProvider nodeIdProvider, ArrayBackedValueStorage abvsFileNode,
-                        TaggedValuePointable nodep,  String nodeId) {
+            ByteBufferInputStream bbis, DataInputStream di, SequenceBuilder sb, ArrayBackedValueStorage abvs,
+            ITreeNodeIdProvider nodeIdProvider, ArrayBackedValueStorage abvsFileNode, TaggedValuePointable nodep,
+            String nodeId) {
         this.args = args;
         this.result = result;
         this.stringp = stringp;
@@ -88,16 +95,21 @@ public class IndexUpdater {
         this.pathsFromFileList = new HashSet<>();
     }
 
-    public void evaluate() throws SystemException, IOException, NoSuchAlgorithmException {
-        String collectionFolder;
-        String indexFolder;
+    /**
+     * Perform the initial configuration for index update/ delete processes.
+     *
+     * @throws SystemException
+     * @throws IOException
+     * @throws NoSuchAlgorithmException
+     */
+    public void setup() throws SystemException, IOException, NoSuchAlgorithmException, JAXBException {
+
         TaggedValuePointable indexTVP = args[0];
 
         if (indexTVP.getTag() != ValueTag.XS_STRING_TAG) {
             throw new SystemException(ErrorCode.FORG0006);
         }
 
-        XmlMetadata collectionMetadata;
         try {
             // Get the index folder
             indexTVP.getValue(stringp);
@@ -107,41 +119,50 @@ public class IndexUpdater {
 
             // Read the metadata file and load the metadata map into memory.
             metaFileUtil = MetaFileUtil.create(indexFolder);
-            metadataMap = metaFileUtil.readMetaFile();
+            metaFileUtil.readMetadataFile();
+            metadataMap = metaFileUtil.getMetadata(indexFolder);
 
             // Retrieve the collection folder path.
             // Remove the entry for ease of the next steps.
-            collectionMetadata = metadataMap.remove(Constants.COLLECTION_ENTRY);
-            collectionFolder = collectionMetadata.getPath();
+            collectionFolder = metaFileUtil.getCollection(indexFolder);
 
         } catch (IOException | ClassNotFoundException e) {
             throw new SystemException(ErrorCode.SYSE0001, e);
         }
 
-        File collectionDirectory = new File(collectionFolder);
-        if (!collectionDirectory.exists()) {
-            throw new RuntimeException("The collection directory (" + collectionFolder + ") does not exist.");
-        }
-
         abvs.reset();
         sb.reset(abvs);
 
         Directory fsdir = FSDirectory.open(Paths.get(indexFolder));
         indexWriter = new IndexWriter(fsdir, new IndexWriterConfig(new CaseSensitiveAnalyzer()).
                 setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND));
+    }
+
+    /**
+     * Wrapper for update index function.
+     *
+     * @throws IOException
+     * @throws NoSuchAlgorithmException
+     */
+    public void updateIndex() throws IOException, NoSuchAlgorithmException {
+        File collectionDirectory = new File(collectionFolder);
+        if (!collectionDirectory.exists()) {
+            throw new RuntimeException("The collection directory (" + collectionFolder + ") does not exist.");
+        }
 
         //Execute update index process
         updateIndex(collectionDirectory);
 
         //Detect deleted files and execute the delete index process.
         deleteIndexOfDeletedFiles(metadataMap.keySet(), pathsFromFileList);
+    }
 
-        // Add collection path entry back
-        metadataMap.put(Constants.COLLECTION_ENTRY, collectionMetadata);
-
-        //Write the updated metadata to the file.
-        metaFileUtil.writeMetaFile(metadataMap);
-
+    /**
+     * Close opened IndexWriter and terminate the index update/ delete process.
+     *
+     * @throws IOException
+     */
+    public void exit() throws IOException {
         indexWriter.forceMerge(1);
 
         indexWriter.close();
@@ -151,6 +172,17 @@ public class IndexUpdater {
     }
 
     /**
+     * Functional wrapper to update Metadata file.
+     *
+     * @throws IOException
+     */
+    public synchronized void updateMetadataFile() throws IOException, JAXBException {
+        //Write the updated metadata to the file.
+        metaFileUtil.updateMetadataMap(metadataMap, indexFolder);
+        metaFileUtil.writeMetadataToFile();
+    }
+
+    /**
      * Check the collection for changes.
      * If changes are detected, update the index
      *
@@ -180,12 +212,14 @@ public class IndexUpdater {
 
                         //Update index corresponding to the xml file.
                         indexWriter.deleteDocuments(new Term(Constants.FIELD_PATH, file.getCanonicalPath()));
-                        indexDocumentBuilder = IndexConstructorUtil.getIndexBuilder(file, indexWriter,
-                                nodep, abvsFileNode, nodeIdProvider, bbis, di, nodeId);
+                        indexDocumentBuilder = IndexConstructorUtil
+                                .getIndexBuilder(file, indexWriter, nodep, abvsFileNode, nodeIdProvider, bbis, di,
+                                        nodeId);
                         indexDocumentBuilder.printStart();
 
-                        if (LOGGER.isDebugEnabled())
+                        if (LOGGER.isDebugEnabled()) {
                             LOGGER.log(Level.DEBUG, "New Index is created for updated file " + file.getCanonicalPath());
+                        }
 
                         //Update the metadata map.
                         XmlMetadata metadata = updateEntry(file, data);
@@ -196,12 +230,13 @@ public class IndexUpdater {
 
                     // In this case, the xml file has not added to the index. (It is a newly added file)
                     // Therefore generate a new index for this file and add it to the existing index.
-                    indexDocumentBuilder = IndexConstructorUtil.getIndexBuilder(file, indexWriter,
-                            nodep, abvsFileNode, nodeIdProvider, bbis, di, nodeId);
+                    indexDocumentBuilder = IndexConstructorUtil
+                            .getIndexBuilder(file, indexWriter, nodep, abvsFileNode, nodeIdProvider, bbis, di, nodeId);
                     indexDocumentBuilder.printStart();
 
-                    if (LOGGER.isDebugEnabled())
+                    if (LOGGER.isDebugEnabled()) {
                         LOGGER.log(Level.DEBUG, "New Index is created for newly added file " + file.getCanonicalPath());
+                    }
 
                     XmlMetadata metadata = updateEntry(file, null);
                     metadataMap.put(file.getCanonicalPath(), metadata);
@@ -212,17 +247,16 @@ public class IndexUpdater {
         }
     }
 
-
     /**
      * Update the current XmlMetadata object related to the currently reading XML file.
      *
-     * @param file : XML file
+     * @param file     : XML file
      * @param metadata : Existing metadata object
      * @return : XML metadata object with updated fields.
      * @throws IOException
      * @throws NoSuchAlgorithmException
      */
-    public XmlMetadata updateEntry(File file, XmlMetadata metadata) throws IOException, NoSuchAlgorithmException {
+    private XmlMetadata updateEntry(File file, XmlMetadata metadata) throws IOException, NoSuchAlgorithmException {
 
         if (metadata == null)
             metadata = new XmlMetadata();
@@ -230,6 +264,7 @@ public class IndexUpdater {
         metadata.setPath(file.getCanonicalPath());
         metadata.setFileName(file.getName());
         metadata.setMd5(metaFileUtil.generateMD5(file));
+        metadata.setLastModified(sdf.format(file.lastModified()));
         return metadata;
     }
 
@@ -240,7 +275,7 @@ public class IndexUpdater {
      * @param pathsFromFileList : Set of paths taken from list of existing files.
      * @throws IOException
      */
-    public void deleteIndexOfDeletedFiles(Set<String> pathsFromMap, Set<String> pathsFromFileList) throws IOException {
+    private void deleteIndexOfDeletedFiles(Set<String> pathsFromMap, Set<String> pathsFromFileList) throws IOException {
         Set<String> sfm = new HashSet<>(pathsFromMap);
 
         // If any file has been deleted from the collection, the number of files stored in metadata is higher  than
@@ -255,10 +290,45 @@ public class IndexUpdater {
             for (String s : sfm) {
                 metadataMap.remove(s);
                 indexWriter.deleteDocuments(new Term(Constants.FIELD_PATH, s));
-                if (LOGGER.isDebugEnabled())
+                if (LOGGER.isDebugEnabled()) {
                     LOGGER.log(Level.DEBUG, "Index of the deleted file " + s + " was deleted from the index!");
+                }
+            }
+        }
+    }
+
+    /**
+     * Delete all indexes in the given directory.
+     * This will also remove the existing metadata file.
+     * It will be created when recreating the index.
+     * When deleting indexes, if any error occurred, the process will be rolled back and all the indexes will be
+     * restored.
+     * Otherwise the changes will be committed.
+     */
+    public void deleteAllIndexes() throws SystemException {
+        try {
+            indexWriter.deleteAll();
+            indexWriter.commit();
+            indexWriter.close();
+            metaFileUtil.deleteMetaDataFile();
+
+            for (File f : (new File(indexFolder)).listFiles())
+                Files.delete(f.toPath());
+
+            sb.finish();
+            result.set(abvs);
+        } catch (IOException e) {
+            try {
+                indexWriter.rollback();
+                indexWriter.close();
+
+                sb.finish();
+                result.set(abvs);
+            } catch (IOException e1) {
+                throw new SystemException(ErrorCode.FOAR0001);
             }
         }
+
     }
 
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/MetaFileUtil.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/MetaFileUtil.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/MetaFileUtil.java
index 97c9da7..53b02df 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/MetaFileUtil.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/MetaFileUtil.java
@@ -19,12 +19,17 @@ package org.apache.vxquery.runtime.functions.index.updateIndex;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
-import javax.xml.bind.DatatypeConverter;
-import java.io.*;
+import javax.xml.bind.*;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -35,6 +40,8 @@ public class MetaFileUtil {
 
     private File metaFile;
     private Logger LOGGER = Logger.getLogger("MetadataFileUtil");
+    private Map<String, ConcurrentHashMap<String, XmlMetadata>> indexes = new ConcurrentHashMap<>();
+    private Map<String, String> indexToCollection = new ConcurrentHashMap<>();
 
     private MetaFileUtil(String indexFolder) {
         this.metaFile = new File(indexFolder + "/" + Constants.META_FILE_NAME);
@@ -46,6 +53,7 @@ public class MetaFileUtil {
 
     /**
      * Checks for existing metadata file.
+     *
      * @return true if the metadata file is present
      */
     public boolean isMetaFilePresent() {
@@ -53,45 +61,98 @@ public class MetaFileUtil {
     }
 
     /**
-     * Write the given List of XmlMetadata objects to a file.
-     * If the metadata file is already presents, delete it.
-     *
-     * @param metadataMap : Set of XmlMetaData objects
+     * Update the content of the metadata map.
+     * If the current collection data is present, replace it.
+     * Otherwise insert new.
+     * @param metadataMap : Set of XmlMetaData objects.
+     * @param index : The path to index location.
      * @throws IOException
      */
-    public void writeMetaFile(ConcurrentHashMap<String, XmlMetadata> metadataMap) throws IOException {
-        if (this.isMetaFilePresent()) Files.delete(Paths.get(metaFile.getCanonicalPath()));
+    public void updateMetadataMap(ConcurrentHashMap<String, XmlMetadata> metadataMap, String index) throws
+            IOException, JAXBException {
 
-        FileOutputStream fileOutputStream = new FileOutputStream(this.metaFile);
-        ObjectOutputStream objectOutputStream = new ObjectOutputStream(fileOutputStream);
-        objectOutputStream.writeObject(metadataMap);
-        objectOutputStream.close();
-
-        if (LOGGER.isDebugEnabled())
-            LOGGER.log(Level.DEBUG, "Writing metadata file completed successfully!");
+        if (this.indexes.get(index) == null) {
+            this.indexes.put(index, metadataMap);
+        } else {
+            this.indexes.replace(index, metadataMap);
+        }
 
     }
 
-
     /**
-     * Read metadata file
+     * Method to get the set of xml metadata for a given collection
      *
-     * @return : List of XmlMetadata objects
+     * @param index : The collection from which the metadata should be read.
+     * @return : Map containing the set of XmlMetadata objects.
      * @throws IOException
      * @throws ClassNotFoundException
      */
-    public ConcurrentHashMap<String, XmlMetadata> readMetaFile() throws IOException, ClassNotFoundException {
-        FileInputStream fin = new FileInputStream(this.metaFile);
-        ObjectInputStream ois = new ObjectInputStream(fin);
-        ConcurrentHashMap<String, XmlMetadata> metadataMap = new ConcurrentHashMap<>((Map<String, XmlMetadata>)ois
-                .readObject()) ;
-        ois.close();
+    public ConcurrentHashMap<String, XmlMetadata> getMetadata(String index)
+            throws IOException, ClassNotFoundException, JAXBException {
+
+        return this.indexes.get(index);
+    }
+
+    /**
+     * Read the metadata file and create an in-memory map containing collection paths and xml files.
+     * @throws JAXBException
+     */
+    public void readMetadataFile() throws JAXBException {
+        JAXBContext jaxbContext = JAXBContext.newInstance(VXQueryIndex.class);
+        Unmarshaller jaxbUnmarshaller = jaxbContext.createUnmarshaller();
+        VXQueryIndex indexes = (VXQueryIndex) jaxbUnmarshaller.unmarshal(metaFile);
+
+        List<XmlMetadataCollection> list = indexes.getIndex();
+
+
+        for (XmlMetadataCollection collection : list) {
+            String indexPath = collection.getIndexLocation();
+            ConcurrentHashMap<String, XmlMetadata> metadataMap = new ConcurrentHashMap<>();
+            List<XmlMetadata> metadata = collection.getMetadataList();
 
-        return metadataMap;
+            this.indexToCollection.put(indexPath, collection.getCollection());
 
+            for (XmlMetadata mData : metadata) {
+                metadataMap.put(mData.getPath(), mData);
+            }
+            this.indexes.put(indexPath, metadataMap);
+        }
     }
 
     /**
+     * Write the content of the ConcurrentHashMap to the xml metadata file.
+     * @throws FileNotFoundException
+     * @throws JAXBException
+     */
+    public void writeMetadataToFile() throws FileNotFoundException, JAXBException {
+        VXQueryIndex index = new VXQueryIndex();
+        List<XmlMetadataCollection> xmlMetadataCollections = new ArrayList<>();
+
+        for (Map.Entry<String, ConcurrentHashMap<String, XmlMetadata>> entry : indexes.entrySet()) {
+            XmlMetadataCollection metadataCollection = new XmlMetadataCollection();
+            List<XmlMetadata> metadataList = new ArrayList<>();
+            metadataCollection.setIndexLocation(entry.getKey());
+            metadataCollection.setCollection(indexToCollection.get(entry.getKey()));
+            metadataList.addAll(entry.getValue().values());
+            metadataCollection.setMetadataList(metadataList);
+            xmlMetadataCollections.add(metadataCollection);
+        }
+        index.setIndex(xmlMetadataCollections);
+
+
+        FileOutputStream fileOutputStream = new FileOutputStream(this.metaFile);
+        JAXBContext jaxbContext = JAXBContext.newInstance(VXQueryIndex.class);
+        Marshaller jaxbMarshaller = jaxbContext.createMarshaller();
+        jaxbMarshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        jaxbMarshaller.marshal(index, fileOutputStream);
+
+        if (LOGGER.isDebugEnabled())
+            LOGGER.log(Level.DEBUG, "Writing metadata file completed successfully!");
+
+    }
+
+
+    /**
      * Generate MD5 checksum string for a given file.
      *
      * @param file : File which the checksum should be generated.
@@ -106,4 +167,43 @@ public class MetaFileUtil {
         return DatatypeConverter.printHexBinary(md5);
     }
 
+    /**
+     * Delete the existing Metadata file.
+     *
+     * @return True if deleted, false otherwise.
+     */
+    public boolean deleteMetaDataFile() {
+        try {
+            Files.delete(Paths.get(metaFile.getCanonicalPath()));
+            if (LOGGER.isDebugEnabled()){
+                LOGGER.log(Level.DEBUG, "Metadata file deleted!");
+            }
+            return true;
+        } catch (IOException e) {
+            if (LOGGER.isTraceEnabled()){
+                LOGGER.log(Level.ERROR, "Metadata file could not be deleted!");
+            }
+            return false;
+        }
+    }
+
+    /**
+     * Get the collection for a given index location.
+     * @param index : path to index
+     * @return
+     */
+    public String getCollection(String index) {
+        return this.indexToCollection.get(index);
+    }
+
+    /**
+     * Set the entry for given index and collection.
+     * @param index : path to index
+     * @param collection : path to corresponding collection
+     */
+    public void setCollectionForIndex(String index, String collection) {
+        if (this.indexToCollection.get(index)==null) {
+            this.indexToCollection.put(index, collection);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/VXQueryIndex.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/VXQueryIndex.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/VXQueryIndex.java
new file mode 100644
index 0000000..fa92b2f
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/VXQueryIndex.java
@@ -0,0 +1,42 @@
+/*
+* 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.vxquery.runtime.functions.index.updateIndex;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.List;
+
+/**
+ * Class for storing metadata information for vxquery index.
+ */
+@XmlAccessorType(XmlAccessType.PROPERTY)
+@XmlRootElement(name = "indexes")
+public class VXQueryIndex {
+
+    private List<XmlMetadataCollection> indexes;
+
+    public List<XmlMetadataCollection> getIndex() {
+        return indexes;
+    }
+
+    @XmlElement(name = "index", type = XmlMetadataCollection.class)
+    public void setIndex(List<XmlMetadataCollection> index) {
+        this.indexes = index;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadata.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadata.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadata.java
index 38f283f..b6da6d9 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadata.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadata.java
@@ -16,20 +16,27 @@
 */
 package org.apache.vxquery.runtime.functions.index.updateIndex;
 
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
 import java.io.Serializable;
 
 /**
- *Class to store metadata related to an XML file.
+ * Class to store metadata related to an XML file.
  * This contains
- *      - Path to the xml file
- *      - MD5 Checksum String
- *      - File name
+ * - Path to the xml file
+ * - MD5 Checksum String
+ * - File name
+ * - Last modified date
  */
-public class XmlMetadata implements Serializable{
+@XmlRootElement(name = "file")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class XmlMetadata implements Serializable {
 
     private String path;
     private String md5;
     private String fileName;
+    private String lastModified;
 
     public String getPath() {
         return path;
@@ -54,4 +61,12 @@ public class XmlMetadata implements Serializable{
     public void setFileName(String fileName) {
         this.fileName = fileName;
     }
+
+    public String getLastModified() {
+        return lastModified;
+    }
+
+    public void setLastModified(String lastModified) {
+        this.lastModified = lastModified;
+    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadataCollection.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadataCollection.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadataCollection.java
new file mode 100644
index 0000000..270fb8d
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/index/updateIndex/XmlMetadataCollection.java
@@ -0,0 +1,62 @@
+/*
+* 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.vxquery.runtime.functions.index.updateIndex;
+
+import javax.xml.bind.annotation.*;
+import java.util.List;
+
+/**
+ * Class for holding the collection information and the list of XML metadata related to the xml files in the
+ * collection.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "index")
+public class XmlMetadataCollection {
+
+    @XmlAttribute(name = "location")
+    private String indexLocation;
+
+    @XmlAttribute(name = "collection")
+    private String collection;
+
+    @XmlElement(name = "file", type = XmlMetadata.class)
+    private List<XmlMetadata> metadataList;
+
+    public List<XmlMetadata> getMetadataList() {
+        return metadataList;
+    }
+
+    public void setMetadataList(List<XmlMetadata> metadataList) {
+        this.metadataList = metadataList;
+    }
+
+    public String getIndexLocation() {
+        return indexLocation;
+    }
+
+    public void setIndexLocation(String indexLocation) {
+        this.indexLocation = indexLocation;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    public void setCollection(String collection) {
+        this.collection = collection;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/test/java/org/apache/vxquery/indexing/MetaFileUtilTest.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/test/java/org/apache/vxquery/indexing/MetaFileUtilTest.java b/vxquery-core/src/test/java/org/apache/vxquery/indexing/MetaFileUtilTest.java
new file mode 100644
index 0000000..6fa92e1
--- /dev/null
+++ b/vxquery-core/src/test/java/org/apache/vxquery/indexing/MetaFileUtilTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.vxquery.indexing;
+
+import junit.framework.Assert;
+import org.apache.commons.io.FileUtils;
+import org.apache.vxquery.runtime.functions.index.updateIndex.MetaFileUtil;
+import org.apache.vxquery.runtime.functions.index.updateIndex.XmlMetadata;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+
+import javax.xml.bind.JAXBException;
+import java.io.File;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Test cases for testing MetaFileUtil functions.
+ * 1) Creating MetaData file
+ * 2) Generating MD5 Hashes
+ * 3) Detecting file changes
+ * 4) Updating metadata
+ * 5) Delete metadata file
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class MetaFileUtilTest {
+
+    private static MetaFileUtil metaFileUtil;
+    private static ConcurrentHashMap<String, XmlMetadata> initialMap;
+    private static ConcurrentHashMap<String, XmlMetadata> modifiedMap;
+
+    @BeforeClass
+    public static void setup() {
+        new File(TestConstants.INDEX_DIR).mkdir();
+        metaFileUtil = MetaFileUtil.create(TestConstants.INDEX_DIR);
+        initialMap = TestConstants.getInitialMap();
+        modifiedMap = TestConstants.getModifiedMap();
+    }
+
+    /**
+     * Test case for generating MD5 string for an XML file.
+     */
+    @Test
+    public void step1_testGenerateMD5ForXML() throws IOException, NoSuchAlgorithmException {
+        TestConstants.createXML("catalog.xml");
+        File xml = new File(TestConstants.XML_FILE);
+        String md5 = metaFileUtil.generateMD5(xml);
+
+        Assert.assertEquals(TestConstants.INITIAL_MD5, md5);
+
+    }
+
+    /**
+     * Test the creation of metadata file.
+     */
+    @Test
+    public void step2_testCreateMetaDataFile() throws IOException, JAXBException {
+        ConcurrentHashMap<String, XmlMetadata> initialMap = TestConstants.getInitialMap();
+        metaFileUtil.updateMetadataMap(initialMap, "");
+        metaFileUtil.writeMetadataToFile();
+        Assert.assertEquals(true, metaFileUtil.isMetaFilePresent());
+    }
+
+    /**
+     * Validate the content of the file.
+     */
+    @Test
+    public void step3_testValidateMetadataFile() throws IOException, ClassNotFoundException, JAXBException {
+        ConcurrentHashMap<String, XmlMetadata> fromFile = metaFileUtil.getMetadata("");
+        Set<String> from = fromFile.keySet();
+        Set<String> initial = initialMap.keySet();
+
+        Assert.assertTrue(from.containsAll(initial));
+
+        for (String key : initial) {
+            Assert.assertEquals(TestConstants.getXMLMetadataString(initialMap.get(key)),
+                    TestConstants.getXMLMetadataString(fromFile.get(key)));
+        }
+
+    }
+
+    /**
+     * Change the xml file and test whether the changes are detected.
+     */
+    @Test
+    public void step4_testDetectFileChanges() throws IOException, NoSuchAlgorithmException {
+        TestConstants.createXML("catalog_edited.xml");
+        File xml = new File(TestConstants.XML_FILE);
+        Assert.assertTrue(metaFileUtil.generateMD5(xml).equals(TestConstants.CHANGED_MD5));
+    }
+
+    /**
+     * Test the update metadata file process.
+     */
+    @Test
+    public void step5_testUpdateMetadata()
+            throws IOException, ClassNotFoundException, NoSuchAlgorithmException, JAXBException {
+        ConcurrentHashMap<String, XmlMetadata> fromFileMap = metaFileUtil.getMetadata("");
+        XmlMetadata modified = fromFileMap.get(TestConstants.XML_FILE);
+
+        File xml = new File(TestConstants.XML_FILE);
+        modified.setMd5(metaFileUtil.generateMD5(xml));
+
+        fromFileMap.replace(TestConstants.XML_FILE, modified);
+
+        metaFileUtil.updateMetadataMap(fromFileMap, TestConstants.INDEX_DIR);
+
+        Assert.assertNotNull(metaFileUtil.getMetadata(TestConstants.INDEX_DIR));
+
+    }
+
+    /**
+     * Validate the updated metadata.
+     */
+    @Test
+    public void step6_testVerifyMetadataChange() throws IOException, ClassNotFoundException, JAXBException {
+        ConcurrentHashMap<String, XmlMetadata> fromFile = metaFileUtil.getMetadata(TestConstants.INDEX_DIR);
+        Set<String> from = fromFile.keySet();
+        Set<String> modified = modifiedMap.keySet();
+
+        Assert.assertTrue(from.containsAll(modified));
+
+        for (String key : modified) {
+            Assert.assertEquals(TestConstants.getXMLMetadataString(modifiedMap.get(key)),
+                    TestConstants.getXMLMetadataString(fromFile.get(key)));
+        }
+    }
+
+    /**
+     * Test deletion of metadata file
+     */
+    @Test
+    public void step7_testDeleteMetadataFile() {
+        metaFileUtil.deleteMetaDataFile();
+        Assert.assertFalse(metaFileUtil.isMetaFilePresent());
+    }
+
+    @AfterClass
+    public static void tearDown() throws IOException {
+        FileUtils.forceDelete(new File(TestConstants.INDEX_DIR));
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/test/java/org/apache/vxquery/indexing/TestConstants.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/test/java/org/apache/vxquery/indexing/TestConstants.java b/vxquery-core/src/test/java/org/apache/vxquery/indexing/TestConstants.java
new file mode 100644
index 0000000..68d40b5
--- /dev/null
+++ b/vxquery-core/src/test/java/org/apache/vxquery/indexing/TestConstants.java
@@ -0,0 +1,113 @@
+/*
+ * 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.vxquery.indexing;
+
+import org.apache.vxquery.runtime.functions.index.updateIndex.XmlMetadata;
+
+import java.io.*;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * TestConstants and methods which will be used in indexing test cases.
+ */
+public class TestConstants {
+    public static String INITIAL_MD5 = "F62EE4BBBBE37183E5F50BB1A0B4FFB4";
+    public static String CHANGED_MD5 = "98B31970B863E86AB2D7852B346FF234";
+
+    public static String COLLECTION = "src/test/resources/collection/";
+    public static String XML_FILE = "/tmp/index/catalog.xml";
+
+    public static String INDEX_DIR = "/tmp/index";
+
+    private static ConcurrentHashMap<String, XmlMetadata> initialMetadataMap = new ConcurrentHashMap<>();
+    private static ConcurrentHashMap<String, XmlMetadata> modifiedMetadataMap = new ConcurrentHashMap<>();
+
+    /**
+     * Creates a HashMap with initial sample data and returns it.
+     *
+     * @return HashMap with sample data.
+     */
+    public static ConcurrentHashMap<String, XmlMetadata> getInitialMap() {
+        XmlMetadata metadata = new XmlMetadata();
+        metadata.setFileName("catalog.xml");
+        metadata.setPath(XML_FILE);
+        metadata.setMd5(INITIAL_MD5);
+        initialMetadataMap.put(XML_FILE, metadata);
+
+        return initialMetadataMap;
+    }
+
+    /**
+     * Creates a HashMap with modified data and returns it.
+     *
+     * @return HashMap with sample data.
+     */
+    public static ConcurrentHashMap<String, XmlMetadata> getModifiedMap() {
+        XmlMetadata metadata = new XmlMetadata();
+        metadata.setFileName("catalog.xml");
+        metadata.setPath(XML_FILE);
+        metadata.setMd5(CHANGED_MD5);
+        modifiedMetadataMap.put(XML_FILE, metadata);
+
+        return modifiedMetadataMap;
+    }
+
+    /**
+     * Generate XML file from given template.
+     *
+     * @param fileName : Template file name
+     * @throws IOException
+     */
+    public static void createXML(String fileName) throws IOException {
+        File collectionDir = new File(COLLECTION);
+
+        String src = collectionDir.getCanonicalPath() + File.separator + fileName;
+        String dest = INDEX_DIR + File.separator + "catalog.xml";
+
+        //Delete any existing file
+        Files.deleteIfExists(Paths.get(dest));
+
+        File in = new File(src);
+        FileInputStream fileInputStream = new FileInputStream(in);
+        BufferedReader reader = new BufferedReader(new InputStreamReader(fileInputStream));
+
+        FileWriter writer = new FileWriter(dest, true);
+        BufferedWriter bufferedWriter = new BufferedWriter(writer);
+
+        String line;
+
+        while ((line = reader.readLine()) != null) {
+            bufferedWriter.write(line);
+            bufferedWriter.newLine();
+        }
+
+        reader.close();
+
+        bufferedWriter.close();
+    }
+
+    /**
+     * Get the XmlMetadata contents as an String.
+     *
+     * @param metadata : XmlMetadata Object
+     * @return String containing metadata
+     */
+    public static String getXMLMetadataString(XmlMetadata metadata) {
+        return String.format("%s %s %s", metadata.getFileName(), metadata.getPath(), metadata.getMd5());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/test/resources/collection/catalog.xml
----------------------------------------------------------------------
diff --git a/vxquery-core/src/test/resources/collection/catalog.xml b/vxquery-core/src/test/resources/collection/catalog.xml
new file mode 100644
index 0000000..9ba57b9
--- /dev/null
+++ b/vxquery-core/src/test/resources/collection/catalog.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" standalone="yes"?>
+<!--
+  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.
+-->
+<catalog>
+    <book id="bk001">
+        <author>Hightower, Kim</author>
+        <title>The First Book</title>
+        <genre>Fiction</genre>
+        <price>44.95</price>
+        <pub_date>2000-10-01</pub_date>
+        <review>An amazing story of nothing.</review>
+    </book>
+
+    <book id="bk002">
+        <author>Oberg, Bruce</author>
+        <title>The Poet's First Poem</title>
+        <genre>Poem</genre>
+        <price>24.95</price>
+        <review>The least poetic poems of the decade.</review>
+    </book>
+
+    <book id="bk003">
+        <author>Nagata, Suanne</author>
+        <title>Becoming Somebody</title>
+        <genre>Biography</genre>
+        <review>A masterpiece of the fine art of gossiping.</review>
+    </book>
+
+</catalog>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-core/src/test/resources/collection/catalog_edited.xml
----------------------------------------------------------------------
diff --git a/vxquery-core/src/test/resources/collection/catalog_edited.xml b/vxquery-core/src/test/resources/collection/catalog_edited.xml
new file mode 100644
index 0000000..ad2d954
--- /dev/null
+++ b/vxquery-core/src/test/resources/collection/catalog_edited.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" standalone="yes"?>
+<!--
+  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.
+-->
+<catalog>
+    <book id="bk001">
+        <author>Hightower, Kim</author>
+        <title>The First Book</title>
+        <genre>Fiction</genre>
+        <price>44.95</price>
+        <pub_date>2000-10-01</pub_date>
+        <review>An amazing story of nothing.</review>
+    </book>
+
+    <book id="bk002">
+        <author>Oberg, Bruce</author>
+        <title>The Poet's First Poem</title>
+        <genre>Poem</genre>
+        <price>24.95</price>
+        <review>The least poetic poems of the decade.</review>
+    </book>
+
+    <book id="bk004">
+        <author>Nagata, Suanne</author>
+        <title>Becoming Somebody</title>
+        <genre>Biography</genre>
+        <review>A masterpiece of the fine art of gossiping.</review>
+    </book>
+
+</catalog>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/createIndex.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/createIndex.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/createIndex.txt
index e69de29..f32a580 100644
--- a/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/createIndex.txt
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/createIndex.txt
@@ -0,0 +1 @@
+true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/deleteIndex.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/deleteIndex.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/deleteIndex.txt
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/deleteIndex.txt
@@ -0,0 +1 @@
+true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/updateIndex.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/updateIndex.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/updateIndex.txt
index e69de29..f32a580 100644
--- a/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/updateIndex.txt
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Indexing/updateIndex.txt
@@ -0,0 +1 @@
+true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-xtest/src/test/resources/Queries/XQuery/Indexing/deleteIndex.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Indexing/deleteIndex.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Indexing/deleteIndex.xq
new file mode 100644
index 0000000..a409d4e
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Indexing/deleteIndex.xq
@@ -0,0 +1,19 @@
+(: Licensed to the Apache Software Foundation (ASF) under one
+   or more contributor license agreements.  See the NOTICE file
+   distributed with this work for additional information
+   regarding copyright ownership.  The ASF licenses this file
+   to you under the Apache License, Version 2.0 (the
+   "License"); you may not use this file except in compliance
+   with the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing,
+   software distributed under the License is distributed on an
+   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+   KIND, either express or implied.  See the License for the
+   specific language governing permissions and limitations
+   under the License. :)
+
+(: Update Lucene Index :)
+delete-index("target/tmp/indexFolder")

http://git-wip-us.apache.org/repos/asf/vxquery/blob/446a1e56/vxquery-xtest/src/test/resources/cat/IndexingQueries.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/IndexingQueries.xml b/vxquery-xtest/src/test/resources/cat/IndexingQueries.xml
index 369dc82..763d8c3 100644
--- a/vxquery-xtest/src/test/resources/cat/IndexingQueries.xml
+++ b/vxquery-xtest/src/test/resources/cat/IndexingQueries.xml
@@ -65,4 +65,9 @@
       <query name="updateIndex" date="2016-06-24"/>
       <output-file compare="Text">updateIndex.txt</output-file>
    </test-case>
+   <test-case name="delete-index" FilePath="Indexing/" Creator="Menaka Jayawardena">
+      <description>Delete the existing index</description>
+      <query name="deleteIndex" date="2016-06-24"/>
+      <output-file compare="Text">deleteIndex.txt</output-file>
+   </test-case>
 </test-group>