http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
index c1a6887..8510e8b 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
@@ -21,12 +21,12 @@ package org.apache.hyracks.storage.am.lsm.common.api;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.api.io.FileReference;
 
 @FunctionalInterface
 public interface IVirtualBufferCacheProvider extends Serializable {
-    List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx, 
IFileSplitProvider fileSplitProvider)
+    List<IVirtualBufferCache> getVirtualBufferCaches(INCServiceContext ctx, 
FileReference fileRef)
             throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
deleted file mode 100644
index 9b24e24..0000000
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.common.dataflow;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IIndex;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-
-public abstract class AbstractLSMIndexDataflowHelper extends 
IndexDataflowHelper {
-
-    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
-
-    protected final double bloomFilterFalsePositiveRate;
-
-    protected final List<IVirtualBufferCache> virtualBufferCaches;
-    protected final ILSMMergePolicy mergePolicy;
-    protected final ILSMIOOperationScheduler ioScheduler;
-    protected final ILSMOperationTrackerFactory opTrackerFactory;
-    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
-    protected final ITypeTraits[] filterTypeTraits;
-    protected final IBinaryComparatorFactory[] filterCmpFactories;
-    protected final int[] filterFields;
-
-    public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, 
IHyracksTaskContext ctx, int partition,
-            List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy 
mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, 
ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ITypeTraits[] 
filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, 
boolean durable)
-            throws HyracksDataException {
-        this(opDesc, ctx, partition, virtualBufferCaches, 
DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackFactory, 
filterTypeTraits, filterCmpFactories, filterFields,
-                durable);
-    }
-
-    public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, 
IHyracksTaskContext ctx, int partition,
-            List<IVirtualBufferCache> virtualBufferCaches, double 
bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory 
opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, 
ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] 
filterCmpFactories, int[] filterFields,
-            boolean durable) throws HyracksDataException {
-        super(opDesc, ctx, partition, durable);
-        this.virtualBufferCaches = virtualBufferCaches;
-        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.mergePolicy = mergePolicy;
-        this.opTrackerFactory = opTrackerFactory;
-        this.ioScheduler = ioScheduler;
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
-        this.filterTypeTraits = filterTypeTraits;
-        this.filterCmpFactories = filterCmpFactories;
-        this.filterFields = filterFields;
-    }
-
-    @Override
-    public int getNumFilterFields() {
-        return filterFields == null ? 0 : filterFields.length;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
deleted file mode 100644
index a95574d..0000000
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.common.dataflow;
-
-import java.util.Map;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import 
org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
-
-public abstract class AbstractLSMIndexDataflowHelperFactory implements 
IIndexDataflowHelperFactory {
-    protected static final long serialVersionUID = 1L;
-
-    protected final IVirtualBufferCacheProvider virtualBufferCacheProvider;
-    protected final ILSMMergePolicyFactory mergePolicyFactory;
-    protected final Map<String, String> mergePolicyProperties;
-    protected final ILSMOperationTrackerFactory opTrackerFactory;
-    protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
-    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
-    protected final double bloomFilterFalsePositiveRate;
-    protected final ITypeTraits[] filterTypeTraits;
-    protected final IBinaryComparatorFactory[] filterCmpFactories;
-    protected final int[] filterFields;
-    protected final boolean durable;
-
-    public AbstractLSMIndexDataflowHelperFactory(IVirtualBufferCacheProvider 
virtualBufferCacheProvider,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> 
mergePolicyProperties,
-            ILSMOperationTrackerFactory opTrackerFactory, 
ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, double 
bloomFilterFalsePositiveRate,
-            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] 
filterCmpFactories, int[] filterFields,
-            boolean durable) {
-        this.virtualBufferCacheProvider = virtualBufferCacheProvider;
-        this.mergePolicyFactory = mergePolicyFactory;
-        this.opTrackerFactory = opTrackerFactory;
-        this.ioSchedulerProvider = ioSchedulerProvider;
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
-        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.mergePolicyProperties = mergePolicyProperties;
-        this.filterTypeTraits = filterTypeTraits;
-        this.filterCmpFactories = filterCmpFactories;
-        this.filterFields = filterFields;
-        this.durable = durable;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
index 48f59f1..5e4bc7d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
@@ -24,18 +24,18 @@ import 
org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import 
org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
 
 public class LSMIndexCompactOperatorNodePushable extends 
AbstractOperatorNodePushable {
     private final IIndexDataflowHelper indexHelper;
 
-    public LSMIndexCompactOperatorNodePushable(IIndexOperatorDescriptor 
opDesc, IHyracksTaskContext ctx, int partition)
-            throws HyracksDataException {
-        this.indexHelper = 
opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, 
partition);
+    public LSMIndexCompactOperatorNodePushable(IHyracksTaskContext ctx, int 
partition,
+            IIndexDataflowHelperFactory indexHelperFactory) throws 
HyracksDataException {
+        this.indexHelper = indexHelperFactory.create(ctx, partition);
     }
 
     @Override
@@ -57,10 +57,10 @@ public class LSMIndexCompactOperatorNodePushable extends 
AbstractOperatorNodePus
     public void initialize() throws HyracksDataException {
         indexHelper.open();
         ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
-        ILSMIndexAccessor accessor = 
index.createAccessor(NoOpOperationCallback.INSTANCE,
-                NoOpOperationCallback.INSTANCE);
+        ILSMIndexAccessor accessor =
+                index.createAccessor(NoOpOperationCallback.INSTANCE, 
NoOpOperationCallback.INSTANCE);
         try {
-            accessor.scheduleFullMerge(NoOpIOOperationCallback.INSTANCE);
+            
accessor.scheduleFullMerge(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback());
         } catch (Exception e) {
             indexHelper.close();
             throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index b51d132..e77b089 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -21,11 +21,13 @@ package org.apache.hyracks.storage.am.lsm.common.dataflow;
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import 
org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import 
org.apache.hyracks.storage.am.common.dataflow.IndexInsertUpdateDeleteOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -42,10 +44,12 @@ public class LSMIndexInsertUpdateDeleteOperatorNodePushable 
extends IndexInsertU
         appender = new FrameTupleAppender(writeBuffer);
     }
 
-    public 
LSMIndexInsertUpdateDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, 
IHyracksTaskContext ctx,
-            int partition, int[] fieldPermutation, IRecordDescriptorProvider 
recordDescProvider, IndexOperation op)
-            throws HyracksDataException {
-        super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, 
op);
+    public LSMIndexInsertUpdateDeleteOperatorNodePushable(IHyracksTaskContext 
ctx, int partition,
+            IIndexDataflowHelperFactory indexHelperFactory, int[] 
fieldPermutation, RecordDescriptor inputRecDesc,
+            IndexOperation op, IModificationOperationCallbackFactory 
modCallbackFactory,
+            ITupleFilterFactory tupleFilterFactory) throws 
HyracksDataException {
+        super(ctx, partition, indexHelperFactory, fieldPermutation, 
inputRecDesc, op, modCallbackFactory,
+                tupleFilterFactory);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
index 9311599..6da4c8f 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
@@ -21,41 +21,26 @@ package org.apache.hyracks.storage.am.lsm.common.dataflow;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import 
org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
-import 
org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import 
org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
-public class LSMTreeIndexCompactOperatorDescriptor extends 
AbstractTreeIndexOperatorDescriptor {
+public class LSMTreeIndexCompactOperatorDescriptor extends 
AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
+    private final IIndexDataflowHelperFactory indexHelperFactory;
 
     public LSMTreeIndexCompactOperatorDescriptor(IOperatorDescriptorRegistry 
spec,
-            IStorageManager storageManager, IIndexLifecycleManagerProvider 
lifecycleManagerProvider,
-            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] 
bloomFilterKeyFields,
-            IIndexDataflowHelperFactory dataflowHelperFactory,
-            IModificationOperationCallbackFactory 
modificationOpCallbackProvider,
-            IPageManagerFactory pageManagerFactory) {
-        super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, 
fileSplitProvider, typeTraits,
-                comparatorFactories, bloomFilterKeyFields, 
dataflowHelperFactory, null, false, false, null,
-                NoOpLocalResourceFactoryProvider.INSTANCE, 
NoOpOperationCallbackFactory.INSTANCE,
-                modificationOpCallbackProvider, pageManagerFactory);
+            IIndexDataflowHelperFactory indexHelperFactory) {
+        super(spec, 0, 0);
+        this.indexHelperFactory = indexHelperFactory;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int 
nPartitions) throws HyracksDataException {
-        return new LSMIndexCompactOperatorNodePushable(this, ctx, partition);
+        return new LSMIndexCompactOperatorNodePushable(ctx, partition, 
indexHelperFactory);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
index f0a5352..fb884f7 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -21,52 +21,44 @@ package org.apache.hyracks.storage.am.lsm.common.dataflow;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import 
org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import 
org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
-import 
org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import 
org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
-public class LSMTreeIndexInsertUpdateDeleteOperatorDescriptor extends 
AbstractTreeIndexOperatorDescriptor {
+public class LSMTreeIndexInsertUpdateDeleteOperatorDescriptor extends 
AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     protected final int[] fieldPermutation;
     protected final IndexOperation op;
+    protected final IIndexDataflowHelperFactory indexHelperFactory;
+    protected final IModificationOperationCallbackFactory modCallbackFactory;
+    protected final ITupleFilterFactory tupleFilterFactory;
 
-    public 
LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry 
spec, RecordDescriptor recDesc,
-            IStorageManager storageManager, IIndexLifecycleManagerProvider 
lifecycleManagerProvider,
-            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] 
bloomFilterKeyFields, int[] fieldPermutation,
-            IndexOperation op, IIndexDataflowHelperFactory 
dataflowHelperFactory,
-            ITupleFilterFactory tupleFilterFactory, IMissingWriterFactory 
missingWriterFactory,
-            IModificationOperationCallbackFactory 
modificationOpCallbackProvider,
-            ISearchOperationCallbackFactory searchOpCallbackProvider, 
IPageManagerFactory pageManagerFactory) {
-        super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, 
fileSplitProvider, typeTraits,
-                comparatorFactories, bloomFilterKeyFields, 
dataflowHelperFactory, tupleFilterFactory, false, false,
-                missingWriterFactory, 
NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
-                modificationOpCallbackProvider, pageManagerFactory);
+    public 
LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry 
spec,
+            RecordDescriptor outRecDesc, IIndexDataflowHelperFactory 
indexHelperFactory, int[] fieldPermutation,
+            IndexOperation op, IModificationOperationCallbackFactory 
modCallbackFactory,
+            ITupleFilterFactory tupleFilterFactory) {
+        super(spec, 1, 1);
+        this.indexHelperFactory = indexHelperFactory;
+        this.modCallbackFactory = modCallbackFactory;
+        this.tupleFilterFactory = tupleFilterFactory;
         this.fieldPermutation = fieldPermutation;
         this.op = op;
+        this.outRecDescs[0] = outRecDesc;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int 
nPartitions) throws HyracksDataException {
-        return new LSMIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, 
partition, fieldPermutation,
-                recordDescProvider, op);
+        return new LSMIndexInsertUpdateDeleteOperatorNodePushable(ctx, 
partition, indexHelperFactory, fieldPermutation,
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 
0), op, modCallbackFactory,
+                tupleFilterFactory);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
new file mode 100644
index 0000000..6255c1d
--- /dev/null
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.dataflow;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import org.apache.hyracks.storage.common.IResource;
+import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.LocalResource;
+
+/**
+ * TODO(amoudi): Change this class and its subclasses to use json 
serialization instead of Java serialization
+ * The base resource that will be written to disk. it will go in the 
serializable resource
+ * member in {@link LocalResource}
+ */
+public abstract class LsmResource implements IResource {
+
+    private static final long serialVersionUID = 1L;
+    protected final String path;
+    protected final IStorageManager storageManager;
+    protected final ITypeTraits[] typeTraits;
+    protected final IBinaryComparatorFactory[] cmpFactories;
+    protected final ITypeTraits[] filterTypeTraits;
+    protected final IBinaryComparatorFactory[] filterCmpFactories;
+    protected final int[] filterFields;
+    protected final ILSMOperationTrackerFactory opTrackerProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    protected final IMetadataPageManagerFactory metadataPageManagerFactory;
+    protected final IVirtualBufferCacheProvider vbcProvider;
+    protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
+    protected final ILSMMergePolicyFactory mergePolicyFactory;
+    protected final Map<String, String> mergePolicyProperties;
+    protected final boolean durable;
+
+    public LsmResource(String path, IStorageManager storageManager, 
ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] 
filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider, 
ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory, 
IVirtualBufferCacheProvider vbcProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, 
ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, boolean durable) {
+        this.path = path;
+        this.storageManager = storageManager;
+        this.typeTraits = typeTraits;
+        this.cmpFactories = cmpFactories;
+        this.filterTypeTraits = filterTypeTraits;
+        this.filterCmpFactories = filterCmpFactories;
+        this.filterFields = filterFields;
+        this.opTrackerProvider = opTrackerProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.metadataPageManagerFactory = metadataPageManagerFactory;
+        this.vbcProvider = vbcProvider;
+        this.ioSchedulerProvider = ioSchedulerProvider;
+        this.mergePolicyFactory = mergePolicyFactory;
+        this.mergePolicyProperties = mergePolicyProperties;
+        this.durable = durable;
+    }
+
+    @Override
+    public String getPath() {
+        return path;
+    }
+
+    public static int getIoDeviceNum(IIOManager ioManager, IODeviceHandle 
deviceHandle) {
+        List<IODeviceHandle> ioDevices = ioManager.getIODevices();
+        for (int i = 0; i < ioDevices.size(); i++) {
+            IODeviceHandle device = ioDevices.get(i);
+            if (device == deviceHandle) {
+                return i;
+            }
+        }
+        return -1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
new file mode 100644
index 0000000..c5634aa
--- /dev/null
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.dataflow;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import 
org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import org.apache.hyracks.storage.common.IResourceFactory;
+import org.apache.hyracks.storage.common.IStorageManager;
+
+public abstract class LsmResourceFactory implements IResourceFactory {
+
+    private static final long serialVersionUID = 1L;
+    protected final IStorageManager storageManager;
+    protected final ITypeTraits[] typeTraits;
+    protected final IBinaryComparatorFactory[] cmpFactories;
+    protected final ITypeTraits[] filterTypeTraits;
+    protected final IBinaryComparatorFactory[] filterCmpFactories;
+    protected final int[] filterFields;
+    protected final ILSMOperationTrackerFactory opTrackerProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    protected final IMetadataPageManagerFactory metadataPageManagerFactory;
+    protected final IVirtualBufferCacheProvider vbcProvider;
+    protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
+    protected final ILSMMergePolicyFactory mergePolicyFactory;
+    protected final Map<String, String> mergePolicyProperties;
+    protected final boolean durable;
+
+    public LsmResourceFactory(IStorageManager storageManager, ITypeTraits[] 
typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] 
filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider, 
ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory, 
IVirtualBufferCacheProvider vbcProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, 
ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, boolean durable) {
+        this.storageManager = storageManager;
+        this.typeTraits = typeTraits;
+        this.cmpFactories = cmpFactories;
+        this.filterTypeTraits = filterTypeTraits;
+        this.filterCmpFactories = filterCmpFactories;
+        this.filterFields = filterFields;
+        this.opTrackerProvider = opTrackerProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.metadataPageManagerFactory = metadataPageManagerFactory;
+        this.vbcProvider = vbcProvider;
+        this.ioSchedulerProvider = ioSchedulerProvider;
+        this.mergePolicyFactory = mergePolicyFactory;
+        this.mergePolicyProperties = mergePolicyProperties;
+        this.durable = durable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index aa23093..df42972 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -34,19 +34,19 @@ import 
org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionTy
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
@@ -317,4 +317,9 @@ public abstract class AbstractLSMIndex implements ILSMIndex 
{
     public List<ILSMMemoryComponent> getMemoryComponents() {
         return memoryComponents;
     }
+
+    @Override
+    public int getNumOfFilterFields() {
+        return filterFields == null ? 0 : filterFields.length;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 767699d..6b9af7e 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -55,7 +55,7 @@ public class ConstantMergePolicy implements ILSMMergePolicy {
 
     @Override
     public void configure(Map<String, String> properties) {
-        numComponents = Integer.parseInt(properties.get("num-components"));
+        numComponents = 
Integer.parseInt(properties.get(ConstantMergePolicyFactory.NUM_COMPONENTS));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
index e75aa7f..52bf896 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
@@ -23,24 +23,15 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.am.common.api.IResourceLifecycleManager;
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
 public class ConstantMergePolicyFactory implements ILSMMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
-
-    private static final String[] SET_VALUES = new String[] { "num-components" 
};
-    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<String>(Arrays.asList(SET_VALUES));
-
-    @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IHyracksTaskContext ctx) {
-        ILSMMergePolicy policy = new ConstantMergePolicy();
-        policy.configure(properties);
-        return policy;
-    }
+    public static final String NUM_COMPONENTS = "num-components";
+    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<>(Arrays.asList(NUM_COMPONENTS));
 
     @Override
     public String getName() {
@@ -53,9 +44,9 @@ public class ConstantMergePolicyFactory implements 
ILSMMergePolicyFactory {
     }
 
     @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IResourceLifecycleManager ilcm) {
+    public ILSMMergePolicy createMergePolicy(Map<String, String> 
configuration, INCServiceContext ctx) {
         ILSMMergePolicy policy = new ConstantMergePolicy();
-        policy.configure(properties);
+        policy.configure(configuration);
         return policy;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
index e6c8186..ee67fbe 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -26,8 +26,6 @@ import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndexCursor;
-import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
@@ -39,6 +37,8 @@ import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
 
 public class ExternalIndexHarness extends LSMHarness {
     private static final Logger LOGGER = 
Logger.getLogger(ExternalIndexHarness.class.getName());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexFactory.java
index 1f7d7a3..55fc92c 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexFactory.java
@@ -22,8 +22,8 @@ package org.apache.hyracks.storage.am.lsm.common.impls;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.common.api.IIndex;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
+import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilter.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilter.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilter.java
index 32937a9..5f01550 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilter.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilter.java
@@ -25,8 +25,8 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.common.MultiComparator;
 
 public class LSMComponentFilter implements ILSMComponentFilter {
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilterManager.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilterManager.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilterManager.java
index 7f8e990..90ca7d6 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilterManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFilterManager.java
@@ -26,11 +26,11 @@ import 
org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import 
org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
-import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterReference;
+import org.apache.hyracks.storage.common.MultiComparator;
 
 public class LSMComponentFilterManager implements ILSMComponentFilterManager {
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 682e323..94fd2bb 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -30,8 +30,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndexCursor;
-import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -47,6 +45,8 @@ import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
 
 public class LSMHarness implements ILSMHarness {
     private static final Logger LOGGER = 
Logger.getLogger(LSMHarness.class.getName());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 58f0c7a..d645876 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -25,10 +25,14 @@ import java.util.PriorityQueue;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
-import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
-import org.apache.hyracks.storage.am.lsm.common.api.*;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 8293f4c..f117e63 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -24,8 +24,6 @@ import java.util.List;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndexCursor;
-import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -34,6 +32,8 @@ import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
 
 public abstract class LSMTreeIndexAccessor implements ILSMIndexAccessor {
     protected ILSMHarness lsmHarness;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
index 052e9af..3195f57 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
@@ -24,8 +24,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.am.common.api.IResourceLifecycleManager;
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
@@ -34,14 +33,7 @@ public class NoMergePolicyFactory implements 
ILSMMergePolicyFactory {
     private static final long serialVersionUID = 1L;
 
     private static final String[] SET_VALUES = new String[] {};
-    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<String>(Arrays.asList(SET_VALUES));
-
-    @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IHyracksTaskContext ctx) {
-        ILSMMergePolicy policy = new NoMergePolicy();
-        policy.configure(properties);
-        return policy;
-    }
+    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<>(Arrays.asList(SET_VALUES));
 
     @Override
     public String getName() {
@@ -54,9 +46,9 @@ public class NoMergePolicyFactory implements 
ILSMMergePolicyFactory {
     }
 
     @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IResourceLifecycleManager ilcm) {
+    public ILSMMergePolicy createMergePolicy(Map<String, String> 
configuration, INCServiceContext ctx) {
         ILSMMergePolicy policy = new NoMergePolicy();
-        policy.configure(properties);
+        policy.configure(configuration);
         return policy;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
deleted file mode 100644
index dc69092..0000000
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-
-public enum NoOpIOOperationCallback
-        implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider, 
ILSMIOOperationCallbackFactory {
-    INSTANCE;
-
-    @Override
-    public void beforeOperation(LSMOperationType opType) throws 
HyracksDataException {
-        // Do nothing.
-    }
-
-    @Override
-    public void afterOperation(LSMOperationType opType, List<ILSMComponent> 
oldComponents,
-            ILSMDiskComponent newComponent) throws HyracksDataException {
-        // Do nothing.
-    }
-
-    @Override
-    public void afterFinalize(LSMOperationType opType, ILSMDiskComponent 
newComponent) throws HyracksDataException {
-        // Do nothing.
-    }
-
-    @Override
-    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
-        return INSTANCE;
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback() {
-        return INSTANCE;
-    }
-
-    @Override
-    public void setNumOfMutableComponents(int count) {
-        // Do nothing.
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
new file mode 100644
index 0000000..bd153a8
--- /dev/null
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+
+public enum NoOpIOOperationCallbackFactory implements 
ILSMIOOperationCallbackFactory {
+    INSTANCE;
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback() {
+        return NoOpIOOperationCallback.INSTANCE;
+    }
+
+    public static class NoOpIOOperationCallback implements 
ILSMIOOperationCallback {
+        private static final NoOpIOOperationCallback INSTANCE = new 
NoOpIOOperationCallback();
+
+        private NoOpIOOperationCallback() {
+
+        }
+
+        @Override
+        public void beforeOperation(LSMOperationType opType) throws 
HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterOperation(LSMOperationType opType, 
List<ILSMComponent> oldComponents,
+                ILSMDiskComponent newComponent) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterFinalize(LSMOperationType opType, ILSMDiskComponent 
newComponent) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void setNumOfMutableComponents(int count) {
+            // Do nothing.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
index c737ceb..55a2164 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
@@ -20,12 +20,12 @@ package org.apache.hyracks.storage.am.lsm.common.impls;
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IModificationOperationCallback;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
 /**
  * Operation tracker that does nothing.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
index e1e05f6..bae551b 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
@@ -24,8 +24,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.am.common.api.IResourceLifecycleManager;
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
@@ -33,16 +32,9 @@ public class PrefixMergePolicyFactory implements 
ILSMMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
 
-    private static final String[] SET_VALUES = new String[] { 
"max-mergable-component-size",
-            "max-tolerance-component-count" };
-    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<String>(Arrays.asList(SET_VALUES));
-
-    @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IHyracksTaskContext ctx) {
-        ILSMMergePolicy policy = new PrefixMergePolicy();
-        policy.configure(properties);
-        return policy;
-    }
+    private static final String[] SET_VALUES =
+            new String[] { "max-mergable-component-size", 
"max-tolerance-component-count" };
+    private static final Set<String> PROPERTIES_NAMES = new 
HashSet<>(Arrays.asList(SET_VALUES));
 
     @Override
     public String getName() {
@@ -55,9 +47,9 @@ public class PrefixMergePolicyFactory implements 
ILSMMergePolicyFactory {
     }
 
     @Override
-    public ILSMMergePolicy createMergePolicy(Map<String, String> properties, 
IResourceLifecycleManager ilcm) {
+    public ILSMMergePolicy createMergePolicy(Map<String, String> 
configuration, INCServiceContext ctx) {
         ILSMMergePolicy policy = new PrefixMergePolicy();
-        policy.configure(properties);
+        policy.configure(configuration);
         return policy;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
index 9fc79b1..823db04 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import 
org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 
@@ -26,7 +26,7 @@ public enum SynchronousSchedulerProvider implements 
ILSMIOOperationSchedulerProv
     INSTANCE;
 
     @Override
-    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
+    public ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx) {
         return SynchronousScheduler.INSTANCE;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index d5a8488..85081a1 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -21,13 +21,13 @@ package org.apache.hyracks.storage.am.lsm.common.impls;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IModificationOperationCallback;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
 public class ThreadCountingTracker implements ILSMOperationTracker {
     private final AtomicInteger threadRefCount;
@@ -59,7 +59,7 @@ public class ThreadCountingTracker implements 
ILSMOperationTracker {
                 && index.hasFlushRequestForCurrentMutableComponent()) {
             ILSMIndexAccessor accessor =
                     index.createAccessor(NoOpOperationCallback.INSTANCE, 
NoOpOperationCallback.INSTANCE);
-            accessor.scheduleFlush(NoOpIOOperationCallback.INSTANCE);
+            
accessor.scheduleFlush(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
index 3a368d4..7258076 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
@@ -23,8 +23,8 @@ import 
org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndex;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.common.IIndex;
 
 public interface IInvertedIndex extends IIndex {
     IInvertedListCursor createInvertedListCursor();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
index f78eda9..86c0bf1 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
@@ -21,9 +21,9 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.api;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
-import org.apache.hyracks.storage.am.common.api.IIndexCursor;
-import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
 
 public interface IInvertedIndexAccessor extends IIndexAccessor {
     public IInvertedListCursor createInvertedListCursor();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
deleted file mode 100644
index 3a902dd..0000000
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.invertedindex.api;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import 
org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-
-public interface IInvertedIndexOperatorDescriptor extends 
IIndexOperatorDescriptor {
-
-    public ITypeTraits[] getInvListsTypeTraits();
-
-    public IBinaryComparatorFactory[] getInvListsComparatorFactories();
-
-    public ITypeTraits[] getTokenTypeTraits();
-
-    public IBinaryComparatorFactory[] getTokenComparatorFactories();
-
-    public IBinaryTokenizerFactory getTokenizerFactory();
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
index 6ab6933..abea6a0 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
@@ -22,7 +22,7 @@ package org.apache.hyracks.storage.am.lsm.invertedindex.api;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
+import org.apache.hyracks.storage.common.MultiComparator;
 
 public interface IInvertedListCursor extends Comparable<IInvertedListCursor> {
     public void reset(int startPageId, int endPageId, int startOff, int 
numElements);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/735532e4/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
deleted file mode 100644
index a39d8e9..0000000
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.invertedindex.dataflow;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import 
org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
-import 
org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import 
org.apache.hyracks.storage.am.common.dataflow.AbstractIndexOperatorDescriptor;
-import 
org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import 
org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
-import 
org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManager;
-import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
-
-public abstract class AbstractLSMInvertedIndexOperatorDescriptor extends 
AbstractIndexOperatorDescriptor implements
-        IInvertedIndexOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    protected final ITypeTraits[] invListsTypeTraits;
-    protected final IBinaryComparatorFactory[] invListComparatorFactories;
-    protected final ITypeTraits[] tokenTypeTraits;
-    protected final IBinaryComparatorFactory[] tokenComparatorFactories;
-    protected final IBinaryTokenizerFactory tokenizerFactory;
-
-    public 
AbstractLSMInvertedIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, 
int inputArity,
-            int outputArity, RecordDescriptor recDesc, IStorageManager 
storageManager,
-            IFileSplitProvider fileSplitProvider, 
IIndexLifecycleManagerProvider lifecycleManagerProvider,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] 
tokenComparatorFactories,
-            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] 
invListComparatorFactories,
-            IBinaryTokenizerFactory tokenizerFactory, 
IIndexDataflowHelperFactory dataflowHelperFactory,
-            ITupleFilterFactory tupleFilterFactory, boolean retainInput, 
boolean retainNull,
-            IMissingWriterFactory nullWriterFactory, 
ILocalResourceFactoryProvider localResourceFactoryProvider,
-            ISearchOperationCallbackFactory searchOpCallbackFactory,
-            IModificationOperationCallbackFactory 
modificationOpCallbackFactory,
-            IPageManagerFactory pageManagerFactory) {
-        super(spec, inputArity, outputArity, recDesc, storageManager, 
lifecycleManagerProvider, fileSplitProvider,
-                dataflowHelperFactory, tupleFilterFactory, retainInput, 
retainNull, nullWriterFactory,
-                localResourceFactoryProvider, searchOpCallbackFactory, 
modificationOpCallbackFactory,
-                pageManagerFactory);
-        this.invListsTypeTraits = invListsTypeTraits;
-        this.invListComparatorFactories = invListComparatorFactories;
-        this.tokenTypeTraits = tokenTypeTraits;
-        this.tokenComparatorFactories = tokenComparatorFactories;
-        this.tokenizerFactory = tokenizerFactory;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getTokenComparatorFactories() {
-        return tokenComparatorFactories;
-    }
-
-    @Override
-    public ITypeTraits[] getTokenTypeTraits() {
-        return tokenTypeTraits;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
-        return invListComparatorFactories;
-    }
-
-    @Override
-    public IBinaryTokenizerFactory getTokenizerFactory() {
-        return tokenizerFactory;
-    }
-
-    @Override
-    public ITypeTraits[] getInvListsTypeTraits() {
-        return invListsTypeTraits;
-    }
-}

Reply via email to