fix parameters
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 2bab291..db6c2c8 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -155,18 +155,18 @@
          * construct bulk-load index operator
          */
         int[] fieldPermutation = new int[] { 0, 1 };
+        int[] keyFields = new int[] { 0 };
         IBinaryComparatorFactory[] indexCmpFactories = new IBinaryComparatorFactory[1];
         indexCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration + 1,
                 WritableComparator.get(vertexIdClass).getClass());
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, secondaryFileSplitProvider, typeTraits, indexCmpFactories,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, getIndexDataflowHelperFactory());
+                fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR, getIndexDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
          * construct local sort operator
          */
-        int[] keyFields = new int[] { 0 };
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
@@ -365,7 +365,8 @@
         IFileSplitProvider secondaryFileSplitProviderWrite = ClusterConfig.getFileSplitProvider(jobId, writeFile);
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderWrite, typeTraits,
-                indexCmpFactories, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, getIndexDataflowHelperFactory());
+                indexCmpFactories, fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR,
+                getIndexDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
index 1dc1529..4dd78c3 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
@@ -42,9 +42,9 @@
     public TreeIndexBulkReLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             IIndexLifecycleManagerProvider lcManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
-            float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
+            int[] bloomFilterFields, float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
         super(spec, 1, 0, null, storageManager, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                fieldPermutation, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                bloomFilterFields, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
                 NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.fieldPermutation = fieldPermutation;
 
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index 2121ede..c68c758 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -84,7 +84,7 @@
         vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(), pageSize,
                 numPagesInMemComponents));
         ioManager = (IOManager) appCtx.getRootContext().getIOManager();
-        lcManager = new IndexLifecycleManager(numPagesInMemComponents * pageSize * 2);
+        lcManager = new IndexLifecycleManager(numPagesInMemComponents * pageSize * 4);
         localResourceRepository = new TransientLocalResourceRepository();
         resourceIdFactory = new ResourceIdFactory(0);
     }