[ASTERIXDB-2540][STO] Optimize Storage Disk I/O
- user model changes: yes. Add a new storage option:
storage.disk.force.bytes (default 16MB),
- storage format changes: no.
- interface changes: yes.
Introduced IPageWriteCallback to LSM indexes
Details:
- Bypass all queuing (from BufferCache and IOManager) for disk writes.
This queuing is unnecessary but destroys fairness among multiple
writers.
- Introduce IPageWriteCallback to control the behavior of disk page
writes. Currently, this interface is used to perform disk forces
regularly for each writer thread.
Change-Id: I1f618dc7c186623e860239b4d97640fe3528e75b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3285
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mblow@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 0681363..a017676 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -31,6 +31,7 @@
import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexPageWriteCallbackFactory;
import org.apache.asterix.common.transactions.Checkpoint;
import org.apache.asterix.dataflow.data.common.AListElementTokenFactory;
import org.apache.asterix.dataflow.data.common.AOrderedListBinaryTokenizerFactory;
@@ -181,6 +182,7 @@
// ILSMOperationTrackerFactory
registeredClasses.put("NoOpIOOperationCallbackFactory", NoOpIOOperationCallbackFactory.class);
registeredClasses.put("LSMBTreeIOOperationCallbackFactory", LSMIndexIOOperationCallbackFactory.class);
+ registeredClasses.put("LSMIndexPageWriteCallbackFactory", LSMIndexPageWriteCallbackFactory.class);
// ILSMIOOperationSchedulerProvider
registeredClasses.put("AppendOnlyLinkedMetadataPageManagerFactory",
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 9a0cd5a..d1b7f0e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -47,6 +47,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -74,6 +75,7 @@
opTrackerFactory = new TestPrimaryIndexOperationTrackerFactory(dataset.getDatasetId());
}
ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
IStorageManager storageManager = storageComponentProvider.getStorageManager();
IMetadataPageManagerFactory metadataPageManagerFactory =
storageComponentProvider.getMetadataPageManagerFactory();
@@ -81,9 +83,9 @@
storageComponentProvider.getIoOperationSchedulerProvider();
AsterixVirtualBufferCacheProvider vbcProvider = new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
return new TestLsmBtreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
- filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields);
+ filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
+ metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
+ true, bloomFilterFields, bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields);
}
private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 6bd73e7..d841b5a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -25,6 +25,7 @@
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
import java.util.function.Function;
@@ -49,7 +50,8 @@
STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(POSITIVE_INTEGER, 8),
STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d),
STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8),
- STORAGE_COMPRESSION_BLOCK(STRING, "none");
+ STORAGE_COMPRESSION_BLOCK(STRING, "none"),
+ STORAGE_DISK_FORCE_BYTES(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(16, MEGABYTE));
private final IOptionType interpreter;
private final Object defaultValue;
@@ -92,6 +94,8 @@
return "The maximum number of datasets that can be concurrently modified";
case STORAGE_COMPRESSION_BLOCK:
return "The default compression scheme for the storage";
+ case STORAGE_DISK_FORCE_BYTES:
+ return "The number of bytes before each disk force (fsync)";
default:
throw new IllegalStateException("NYI: " + this);
}
@@ -198,4 +202,8 @@
private long getMetadataReservedMemory() {
return (getMetadataMemoryComponentNumPages() * (long) getMemoryComponentPageSize()) * getMetadataDatasets();
}
+
+ public int getDiskForcePages() {
+ return (int) (accessor.getLong(Option.STORAGE_DISK_FORCE_BYTES) / getBufferCachePageSize());
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexPageWriteCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexPageWriteCallbackFactory.java
new file mode 100644
index 0000000..27f18cf
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexPageWriteCallbackFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.ioopcallbacks;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexPageWriteCallback;
+import org.apache.hyracks.storage.common.IResource;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class LSMIndexPageWriteCallbackFactory implements ILSMPageWriteCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ protected transient int pagesPerForce;
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) throws HyracksDataException {
+ INcApplicationContext appCtx = (INcApplicationContext) ncCtx.getApplicationContext();
+ pagesPerForce = appCtx.getStorageProperties().getDiskForcePages();
+ }
+
+ @Override
+ public IPageWriteCallback createPageWriteCallback() throws HyracksDataException {
+ return new LSMIndexPageWriteCallback(pagesPerForce);
+ }
+
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+ return json;
+ }
+
+ @SuppressWarnings("squid:S1172") // unused parameter
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ throws HyracksDataException {
+ return new LSMIndexPageWriteCallbackFactory();
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 1ce9002..7c5902c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -36,6 +36,7 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexPageWriteCallbackFactory;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
@@ -82,6 +83,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
@@ -329,18 +331,21 @@
DatasetInfoProvider datasetInfoProvider = new DatasetInfoProvider(datasetId);
ILSMIOOperationCallbackFactory ioOpCallbackFactory =
new LSMIndexIOOperationCallbackFactory(idGeneratorProvider, datasetInfoProvider);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = new LSMIndexPageWriteCallbackFactory();
+
IStorageComponentProvider storageComponentProvider = appContext.getStorageComponentProvider();
if (isNewUniverse()) {
final double bloomFilterFalsePositiveRate =
appContext.getStorageProperties().getBloomFilterFalsePositiveRate();
- LSMBTreeLocalResourceFactory lsmBtreeFactory = new LSMBTreeLocalResourceFactory(
- storageComponentProvider.getStorageManager(), typeTraits, cmpFactories, null, null, null,
- opTrackerFactory, ioOpCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
- new AsterixVirtualBufferCacheProvider(datasetId),
- storageComponentProvider.getIoOperationSchedulerProvider(),
- appContext.getMetadataMergePolicyFactory(), StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES,
- true, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
- NoOpCompressorDecompressorFactory.INSTANCE);
+ LSMBTreeLocalResourceFactory lsmBtreeFactory =
+ new LSMBTreeLocalResourceFactory(storageComponentProvider.getStorageManager(), typeTraits,
+ cmpFactories, null, null, null, opTrackerFactory, ioOpCallbackFactory,
+ pageWriteCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
+ new AsterixVirtualBufferCacheProvider(datasetId),
+ storageComponentProvider.getIoOperationSchedulerProvider(),
+ appContext.getMetadataMergePolicyFactory(),
+ StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
+ bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE);
DatasetLocalResourceFactory dsLocalResourceFactory =
new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
// TODO(amoudi) Creating the index should be done through the same code path as
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 21fafaf6a..a03e6e6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -50,6 +50,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -75,6 +76,7 @@
double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
IStorageManager storageManager = storageComponentProvider.getStorageManager();
IMetadataPageManagerFactory metadataPageManagerFactory =
storageComponentProvider.getMetadataPageManagerFactory();
@@ -85,13 +87,13 @@
return index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))
? new ExternalBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
- ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+ ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
bloomFilterFalsePositiveRate, false, btreeFields)
: new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
- ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+ ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
bloomFilterFalsePositiveRate, false, btreeFields);
case INTERNAL:
AsterixVirtualBufferCacheProvider vbcProvider =
@@ -107,9 +109,9 @@
return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
- mergePolicyProperties, true, bloomFilterFields, bloomFilterFalsePositiveRate,
- index.isPrimaryIndex(), btreeFields, compDecompFactory);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
+ bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index f7ac890..2be70702a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -36,6 +36,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexPageWriteCallbackFactory;
import org.apache.asterix.common.metadata.IDataset;
import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
import org.apache.asterix.common.utils.JobUtils;
@@ -107,6 +108,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -495,6 +497,10 @@
return new LSMIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
}
+ public ILSMPageWriteCallbackFactory getPageWriteCallbackFactory() throws AlgebricksException {
+ return new LSMIndexPageWriteCallbackFactory();
+ }
+
/**
* get the IndexOperationTrackerFactory for a particular index on the dataset
*
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index fe67e64..b763954 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -47,6 +47,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
@@ -105,6 +106,7 @@
IStorageManager storageManager = storageComponentProvider.getStorageManager();
ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
IMetadataPageManagerFactory metadataPageManagerFactory =
storageComponentProvider.getMetadataPageManagerFactory();
AsterixVirtualBufferCacheProvider vbcProvider = new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -120,10 +122,10 @@
IBinaryTokenizerFactory tokenizerFactory = getTokenizerFactory(dataset, index, recordType, metaType);
return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
- true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, isPartitioned, invertedIndexFields,
- secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
- bloomFilterFalsePositiveRate);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ isPartitioned, invertedIndexFields, secondaryFilterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
}
private static ITypeTraits[] getInvListTypeTraits(MetadataProvider metadataProvider, Dataset dataset,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index a1fb866..2108a4c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -51,6 +51,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterLocalResourceFactory;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -138,6 +139,7 @@
IStorageManager storageManager = storageComponentProvider.getStorageManager();
ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
IMetadataPageManagerFactory metadataPageManagerFactory =
storageComponentProvider.getMetadataPageManagerFactory();
ILSMIOOperationSchedulerProvider ioSchedulerProvider =
@@ -157,15 +159,15 @@
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
return new LSMRTreeWithAntiMatterLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
- mergePolicyProperties, true, valueProviderFactories, rTreePolicyType, linearizeCmpFactory,
- rtreeFields, isPointMBR, btreeCompFactories);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, true, valueProviderFactories, rTreePolicyType,
+ linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories);
} else {
return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
- metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true,
- btreeCompFactories, valueProviderFactories, rTreePolicyType, linearizeCmpFactory, rtreeFields,
- new int[] { numNestedSecondaryKeyFields }, isPointMBR,
+ pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
+ mergePolicyProperties, true, btreeCompFactories, valueProviderFactories, rTreePolicyType,
+ linearizeCmpFactory, rtreeFields, new int[] { numNestedSecondaryKeyFields }, isPointMBR,
mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index b8cf066..46e4fb2 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
@@ -44,9 +45,10 @@
IBinaryComparatorFactory[] filtercmpfactories) {
return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, comparatorFactories, filtertypetraits,
filtercmpfactories, filterfields, ThreadCountingOperationTrackerFactory.INSTANCE,
- NoOpIOOperationCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
- SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
- bloomFilterKeyFields, LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true,
- btreefields, NoOpCompressorDecompressorFactory.INSTANCE);
+ NoOpIOOperationCallbackFactory.INSTANCE, NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory,
+ getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
+ MERGE_POLICY_PROPERTIES, DURABLE, bloomFilterKeyFields,
+ LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, btreefields,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index d9358bd..3b3fc1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeLocalResourceFactory;
@@ -50,9 +51,9 @@
IBinaryComparatorFactory[] secondaryComparatorFactories, IMetadataPageManagerFactory pageManagerFactory) {
return new LSMRTreeLocalResourceFactory(storageManager, secondaryTypeTraits, secondaryComparatorFactories, null,
null, null, ThreadCountingOperationTrackerFactory.INSTANCE, NoOpIOOperationCallbackFactory.INSTANCE,
- pageManagerFactory, getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE,
- MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE, btreeComparatorFactories,
- valueProviderFactories, rtreePolicyType, linearizerCmpFactory, null, btreeFields, IS_POINT_MBR,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+ NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
+ SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
+ btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizerCmpFactory, null,
+ btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index 5b091a2..0bb483a 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterLocalResourceFactory;
@@ -50,9 +51,9 @@
IBinaryComparatorFactory[] secondaryComparatorFactories, IMetadataPageManagerFactory pageManagerFactory) {
return new LSMRTreeWithAntiMatterLocalResourceFactory(storageManager, secondaryTypeTraits,
secondaryComparatorFactories, null, null, null, ThreadCountingOperationTrackerFactory.INSTANCE,
- NoOpIOOperationCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
- SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
- valueProviderFactories, rtreePolicyType, linearizerCmpFactory, btreeFields, IS_POINT_MBR,
- btreeComparatorFactories);
+ NoOpIOOperationCallbackFactory.INSTANCE, NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory,
+ getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
+ MERGE_POLICY_PROPERTIES, DURABLE, valueProviderFactories, rtreePolicyType, linearizerCmpFactory,
+ btreeFields, IS_POINT_MBR, btreeComparatorFactories);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index 2dce1cd..062b1c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -28,7 +28,8 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -271,9 +272,9 @@
bufferCache.deleteFile(file);
}
- public IIndexBulkLoader createBuilder(long numElements, int numHashes, int numBitsPerElement)
- throws HyracksDataException {
- return new BloomFilterBuilder(numElements, numHashes, numBitsPerElement);
+ public IIndexBulkLoader createBuilder(long numElements, int numHashes, int numBitsPerElement,
+ IPageWriteCallback callback) throws HyracksDataException {
+ return new BloomFilterBuilder(numElements, numHashes, numBitsPerElement, callback);
}
public class BloomFilterBuilder extends PageWriteFailureCallback implements IIndexBulkLoader {
@@ -283,17 +284,17 @@
private final long numBits;
private final int numPages;
private long actualNumElements;
- private final IFIFOPageQueue queue;
+ private final IFIFOPageWriter pageWriter;
private final ICachedPage[] pages;
private ICachedPage metaDataPage = null;
@SuppressWarnings("squid:S1181") // Catch Throwable Must return all confiscated pages
- public BloomFilterBuilder(long estimatedNumElemenets, int numHashes, int numBitsPerElement)
- throws HyracksDataException {
+ public BloomFilterBuilder(long estimatedNumElemenets, int numHashes, int numBitsPerElement,
+ IPageWriteCallback callback) throws HyracksDataException {
if (!isActivated) {
throw HyracksDataException.create(ErrorCode.CANNOT_CREATE_BLOOM_FILTER_BUILDER_FOR_INACTIVE_FILTER);
}
- queue = bufferCache.createFIFOQueue();
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
this.estimatedNumElements = estimatedNumElemenets;
this.numHashes = numHashes;
numBits = this.estimatedNumElements * numBitsPerElement;
@@ -377,11 +378,10 @@
@Override
public void end() throws HyracksDataException {
allocateAndInitMetaDataPage();
- queue.put(metaDataPage, this);
+ pageWriter.write(metaDataPage);
for (ICachedPage p : pages) {
- queue.put(p, this);
+ pageWriter.write(p);
}
- bufferCache.finishQueue();
if (hasFailed()) {
throw HyracksDataException.create(getFailure());
}
@@ -403,5 +403,10 @@
bufferCache.returnPage(metaDataPage, false);
}
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 5d2cd26..744f25d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -64,6 +64,7 @@
import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public class BTree extends AbstractTreeIndex {
@@ -992,16 +993,17 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
- return new BTreeBulkLoader(fillFactor, verifyInput);
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
+ return new BTreeBulkLoader(fillFactor, verifyInput, callback);
}
public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
protected final ISplitKey splitKey;
protected final boolean verifyInput;
- public BTreeBulkLoader(float fillFactor, boolean verifyInput) throws HyracksDataException {
- super(fillFactor);
+ public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
+ throws HyracksDataException {
+ super(fillFactor, callback);
this.verifyInput = verifyInput;
splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
@@ -1043,9 +1045,9 @@
((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
- putInQueue(leafFrontier.page);
+ write(leafFrontier.page);
for (ICachedPage c : pagesToWrite) {
- putInQueue(c);
+ write(c);
}
pagesToWrite.clear();
splitKey.setRightPage(leafFrontier.pageId);
@@ -1152,7 +1154,7 @@
ICachedPage lastLeaf = nodeFrontiers.get(level).page;
int lastLeafPage = nodeFrontiers.get(level).pageId;
lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
- putInQueue(lastLeaf);
+ write(lastLeaf);
nodeFrontiers.get(level).page = null;
persistFrontiers(level + 1, lastLeafPage);
return;
@@ -1167,7 +1169,7 @@
((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
int finalPageId = freePageManager.takePage(metaFrame);
frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- putInQueue(frontier.page);
+ write(frontier.page);
frontier.pageId = finalPageId;
persistFrontiers(level + 1, finalPageId);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
index f8a929d..208d3b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
@@ -46,7 +46,7 @@
*
* @throws HyracksDataException
*/
- void close(IPageWriteFailureCallback callback) throws HyracksDataException;
+ void close(IPageWriteFailureCallback failureCallback) throws HyracksDataException;
/**
* Create a metadata frame to be used for reading and writing to metadata pages
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
index 90fe22d..610c15f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.common.IIndex;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
public class IndexBulkLoadOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
protected final IHyracksTaskContext ctx;
@@ -134,6 +135,7 @@
}
protected void initializeBulkLoader() throws HyracksDataException {
- bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ NoOpPageWriteCallback.INSTANCE);
}
-}
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
index 7d43ed8..45fbd1b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
@@ -31,8 +31,9 @@
import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -209,9 +210,9 @@
}
@Override
- public void close(IPageWriteFailureCallback callback) throws HyracksDataException {
+ public void close(IPageWriteFailureCallback failureCallback) throws HyracksDataException {
if (ready) {
- IFIFOPageQueue queue = bufferCache.createFIFOQueue();
+ IFIFOPageWriter pageWriter = bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, failureCallback);
ITreeIndexMetadataFrame metaFrame = frameFactory.createFrame();
confiscatedPage.acquireWriteLatch();
try {
@@ -226,8 +227,7 @@
compressedPageWriter.prepareWrite(confiscatedPage);
// WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page
// won't be flushed to disk because it won't be dirty until the write latch has been released.
- queue.put(confiscatedPage, callback);
- bufferCache.finishQueue();
+ pageWriter.write(confiscatedPage);
compressedPageWriter.endWriting();
metadataPage = getMetadataPageId();
ready = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index f83a27d..3158b79 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -37,7 +37,8 @@
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -242,16 +243,16 @@
// HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
// that tree are allowed. Currently, this is not enforced.
protected boolean releasedLatches;
- private final IFIFOPageQueue queue;
+ private final IFIFOPageWriter pageWriter;
protected List<ICachedPage> pagesToWrite;
private final ICompressedPageWriter compressedPageWriter;
- public AbstractTreeIndexBulkLoader(float fillFactor) throws HyracksDataException {
+ public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
leafFrame = leafFrameFactory.createFrame();
interiorFrame = interiorFrameFactory.createFrame();
metaFrame = freePageManager.createMetadataFrame();
- queue = bufferCache.createFIFOQueue();
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
if (!isEmptyTree(leafFrame)) {
throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
@@ -303,7 +304,6 @@
if (hasFailed()) {
throw HyracksDataException.create(getFailure());
}
- bufferCache.finishQueue();
freePageManager.setRootPageId(rootPage);
}
@@ -325,9 +325,14 @@
this.leafFrame = leafFrame;
}
- public void putInQueue(ICachedPage cPage) throws HyracksDataException {
+ public void write(ICachedPage cPage) throws HyracksDataException {
compressedPageWriter.prepareWrite(cPage);
- queue.put(cPage, this);
+ pageWriter.write(cPage);
+ }
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index 89ccfed..e579f6b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -51,12 +52,14 @@
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
- btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null,
- ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
+ btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
+ metadataPageManagerFactory, null, ioSchedulerProvider, durable,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
@@ -70,11 +73,13 @@
IIOManager ioManager = serviceCtx.getIoManager();
FileReference file = ioManager.resolve(path);
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
return LSMBTreeUtil.createExternalBTree(ioManager, file, storageManager.getBufferCache(serviceCtx), typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
- ioOpCallbackFactory, durable, metadataPageManagerFactory, serviceCtx.getTracer());
+ ioOpCallbackFactory, pageWriteCallbackFactory, durable, metadataPageManagerFactory,
+ serviceCtx.getTracer());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index 555f641..4477ba46 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -28,6 +28,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -40,14 +41,15 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
+ bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
}
@Override
@@ -55,6 +57,7 @@
return new ExternalBTreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ ioSchedulerProvider, durable);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index ddf0955..7753446 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -54,12 +55,14 @@
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable) {
super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
- filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null,
- ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
+ filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
+ metadataPageManagerFactory, null, ioSchedulerProvider, durable,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
@@ -74,11 +77,13 @@
IIOManager ioManager = serviceCtx.getIoManager();
FileReference file = ioManager.resolve(path);
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
return LSMBTreeUtil.createExternalBTreeWithBuddy(ioManager, file, storageManager.getBufferCache(serviceCtx),
typeTraits, cmpFactories, bloomFilterFalsePositiveRate,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
- ioOpCallbackFactory, bloomFilterKeyFields, durable, metadataPageManagerFactory, serviceCtx.getTracer());
+ ioOpCallbackFactory, pageWriteCallbackFactory, bloomFilterKeyFields, durable,
+ metadataPageManagerFactory, serviceCtx.getTracer());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 89e5154..119a31b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -28,6 +28,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -40,14 +41,15 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields, bloomFilterFalsePositiveRate,
- isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
+ bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
}
@Override
@@ -55,6 +57,7 @@
return new ExternalBTreeWithBuddyLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ ioSchedulerProvider, durable);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 3074e9e..52d00f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -38,6 +38,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
@@ -64,12 +65,13 @@
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
ICompressorDecompressorFactory compressorDecompressorFactory) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.isPrimary = isPrimary;
@@ -94,14 +96,15 @@
FileReference file = ioManager.resolve(path);
List<IVirtualBufferCache> vbcs = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
//TODO: enable updateAwareness for secondary LSMBTree indexes
boolean updateAware = false;
return LSMBTreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx), typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
- ioOpCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
- durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer(),
+ ioOpCallbackFactory, pageWriteCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories,
+ btreeFields, filterFields, durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer(),
compressorDecompressorFactory);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index b3d61e8..e71a3c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -30,6 +30,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
@@ -50,14 +51,15 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
ICompressorDecompressorFactory compressorDecompressorFactory) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.isPrimary = isPrimary;
@@ -70,7 +72,8 @@
return new LSMBTreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable, compressorDecompressorFactory);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
+ compressorDecompressorFactory);
}
private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
index 2be5125..0308ef1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -48,6 +48,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
@@ -61,6 +62,7 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.util.trace.ITracer;
/**
@@ -94,11 +96,11 @@
ILSMDiskComponentFactory bulkLoadComponentFactory, ILSMDiskComponentFactory transactionComponentFactory,
double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean durable, ITracer tracer)
- throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, bufferCache, fileManager, componentFactory,
bulkLoadComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, false, durable, tracer);
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, false, durable, tracer);
this.transactionComponentFactory = transactionComponentFactory;
this.secondDiskComponents = new LinkedList<>();
this.interiorFrameFactory = interiorFrameFactory;
@@ -443,8 +445,9 @@
loadOp.setNewComponent(component);
ioOpCallback.scheduled(loadOp);
ioOpCallback.beforeOperation(loadOp);
- componentBulkLoader =
- component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, true);
+ IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
+ componentBulkLoader = component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false,
+ true, true, pageWriteCallback);
}
// It is expected that the mode was set to insert operation before
@@ -509,6 +512,11 @@
public Throwable getFailure() {
return componentBulkLoader.getFailure();
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ componentBulkLoader.force();
+ }
}
// The accessor for disk only indexes don't use modification callback and always carry the target index version with them
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index be91244..d207b94 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -52,6 +52,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
@@ -89,11 +90,12 @@
IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
ILSMDiskComponentFactory bulkLoadComponentFactory, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, IBinaryComparatorFactory[] btreeCmpFactories,
- IBinaryComparatorFactory[] buddyBtreeCmpFactories, int[] buddyBTreeFields, boolean durable, ITracer tracer)
- throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ IBinaryComparatorFactory[] btreeCmpFactories, IBinaryComparatorFactory[] buddyBtreeCmpFactories,
+ int[] buddyBTreeFields, boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, componentFactory, bulkLoadComponentFactory, durable, tracer);
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, bulkLoadComponentFactory,
+ durable, tracer);
this.btreeCmpFactories = btreeCmpFactories;
this.buddyBtreeCmpFactories = buddyBtreeCmpFactories;
this.buddyBTreeFields = buddyBTreeFields;
@@ -334,8 +336,8 @@
numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
.getBloomFilter().getNumElements();
}
- componentBulkLoader =
- mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+ componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false,
+ false, pageWriteCallbackFactory.createPageWriteCallback());
try {
while (buddyBtreeCursor.hasNext()) {
buddyBtreeCursor.next();
@@ -346,7 +348,8 @@
buddyBtreeCursor.close();
}
} else {
- componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
+ componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false,
+ pageWriteCallbackFactory.createPageWriteCallback());
}
try {
@@ -527,8 +530,8 @@
loadOp.setNewComponent(component);
ioOpCallback.scheduled(loadOp);
ioOpCallback.beforeOperation(loadOp);
- componentBulkLoader =
- component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
+ componentBulkLoader = component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false,
+ true, false, pageWriteCallbackFactory.createPageWriteCallback());
}
@Override
@@ -593,6 +596,11 @@
public Throwable getFailure() {
return componentBulkLoader.getFailure();
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ componentBulkLoader.force();
+ }
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 061c0ef..c041b06 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -56,6 +56,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
@@ -70,6 +71,7 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.util.trace.ITracer;
public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
@@ -94,12 +96,13 @@
ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, boolean hasBloomFilter,
- int[] btreeFields, int[] filterFields, boolean durable, boolean updateAware, ITracer tracer)
- throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean needKeyDupCheck, boolean hasBloomFilter, int[] btreeFields, int[] filterFields, boolean durable,
+ boolean updateAware, ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
- opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, bulkLoadComponentFactory,
- filterFrameFactory, filterManager, filterFields, durable, filterHelper, btreeFields, tracer);
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+ bulkLoadComponentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
+ btreeFields, tracer);
this.insertLeafFrameFactory = insertLeafFrameFactory;
this.deleteLeafFrameFactory = deleteLeafFrameFactory;
this.cmpFactories = cmpFactories;
@@ -124,10 +127,11 @@
ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, boolean durable,
- ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean needKeyDupCheck, boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, bufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackFactory, componentFactory, bulkLoadComponentFactory, durable, tracer);
+ ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, bulkLoadComponentFactory, durable,
+ tracer);
this.insertLeafFrameFactory = insertLeafFrameFactory;
this.deleteLeafFrameFactory = deleteLeafFrameFactory;
this.cmpFactories = cmpFactories;
@@ -277,7 +281,8 @@
}
component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
true);
- componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false,
+ pageWriteCallbackFactory.createPageWriteCallback());
IIndexCursor scanCursor = accessor.createSearchCursor(false);
accessor.search(scanCursor, nullPred);
try {
@@ -336,8 +341,9 @@
long numElements = getNumberOfElements(mergedComponents);
mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
mergeOp.getBloomFilterTarget(), true);
- componentBulkLoader =
- mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+ IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
+ componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
+ false, false, pageWriteCallback);
while (cursor.hasNext()) {
cursor.next();
ITupleReference frameTuple = cursor.getTuple();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index 5aaf0e2..8c5cdbb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -52,6 +52,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.frames.LSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -71,10 +72,11 @@
FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits,
- IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields, boolean durable,
- IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
- ICompressorDecompressorFactory compressorDecompressorFactory) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
+ boolean updateAware, ITracer tracer, ICompressorDecompressorFactory compressorDecompressorFactory)
+ throws HyracksDataException {
LSMBTreeTupleWriterFactory insertTupleWriterFactory =
new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
@@ -127,16 +129,17 @@
return new LSMBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
- cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, needKeyDupCheck, hasBloomFilter,
- btreeFields, filterFields, durable, updateAware, tracer);
+ cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
+ needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer);
}
public static ExternalBTree createExternalBTree(IIOManager ioManager, FileReference file,
IBufferCache diskBufferCache, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean durable,
- IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean durable, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
+ throws HyracksDataException {
LSMBTreeTupleWriterFactory insertTupleWriterFactory =
new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, false);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
@@ -176,15 +179,15 @@
return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
transactionComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, durable, tracer);
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, durable, tracer);
}
public static ExternalBTreeWithBuddy createExternalBTreeWithBuddy(IIOManager ioManager, FileReference file,
IBufferCache diskBufferCache, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- int[] buddyBTreeFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
- throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] buddyBTreeFields, boolean durable,
+ IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer) throws HyracksDataException {
ITypeTraits[] buddyBtreeTypeTraits = new ITypeTraits[buddyBTreeFields.length];
IBinaryComparatorFactory[] buddyBtreeCmpFactories = new IBinaryComparatorFactory[buddyBTreeFields.length];
for (int i = 0; i < buddyBtreeTypeTraits.length; i++) {
@@ -233,6 +236,7 @@
return new ExternalBTreeWithBuddy(ioManager, interiorFrameFactory, insertLeafFrameFactory,
buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
bulkLoadComponentFactory, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackFactory, cmpFactories, buddyBtreeCmpFactories, buddyBTreeFields, durable, tracer);
+ ioOpCallbackFactory, pageWriteCallbackFactory, cmpFactories, buddyBtreeCmpFactories, buddyBTreeFields,
+ durable, tracer);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
index 6b6e5e0..db36c72 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
public abstract class AbstractLSMWithBloomFilterDiskComponent extends AbstractLSMDiskComponent {
@@ -77,21 +78,22 @@
getBloomFilter().purge();
}
- public IChainedComponentBulkLoader createBloomFilterBulkLoader(long numElementsHint) throws HyracksDataException {
+ public IChainedComponentBulkLoader createBloomFilterBulkLoader(long numElementsHint, IPageWriteCallback callback)
+ throws HyracksDataException {
BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(
BloomCalculations.maxBucketsPerElement(numElementsHint), getLsmIndex().bloomFilterFalsePositiveRate());
return new BloomFilterBulkLoader(getBloomFilter().createBuilder(numElementsHint, bloomFilterSpec.getNumHashes(),
- bloomFilterSpec.getNumBucketsPerElements()));
+ bloomFilterSpec.getNumBucketsPerElements(), callback));
}
@Override
public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
- boolean cleanupEmptyComponent) throws HyracksDataException {
+ boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
- numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
+ numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent, callback);
if (numElementsHint > 0) {
- chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
+ chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint, callback));
}
return chainedBulkLoader;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
index f7feb78..fda48f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
public abstract class AbstractLSMWithBuddyDiskComponent extends AbstractLSMWithBloomFilterDiskComponent {
@@ -77,12 +78,12 @@
}
@Override
- public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+ protected IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
IIndexBulkLoader indexBulkLoader =
- getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
IIndexBulkLoader buddyBulkLoader =
- getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
index 543779c..029a59d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
@@ -23,9 +23,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
public interface ILSMDiskComponent extends ILSMComponent {
@@ -122,27 +121,6 @@
void validate() throws HyracksDataException;
/**
- * Creates a chained bulkloader which populates component's LSM filter
- *
- * @return
- * @throws HyracksDataException
- */
- IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException;
-
- /**
- * Creates a chained bulkloader which populates component's index
- *
- * @param fillFactor
- * @param verifyInput
- * @param numElementsHint
- * @param checkIfEmptyIndex
- * @return
- * @throws HyracksDataException
- */
- IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException;
-
- /**
* Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
* component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
*
@@ -153,10 +131,10 @@
* @param checkIfEmptyIndex
* @param withFilter
* @param cleanupEmptyComponent
- * @return
+ * @return the created disk component bulk loader
* @throws HyracksDataException
*/
- ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
- throws HyracksDataException;
+ ILSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent,
+ IPageWriteCallback callback) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMPageWriteCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMPageWriteCallbackFactory.java
new file mode 100644
index 0000000..bd92eaf
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMPageWriteCallbackFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.common.IResource;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public interface ILSMPageWriteCallbackFactory extends Serializable, IJsonSerializable {
+ /**
+ * Initialize the callback factory with the given ncCtx and resource
+ *
+ * @param ncCtx
+ */
+ void initialize(INCServiceContext ncCtx, IResource resource) throws HyracksDataException;
+
+ /**
+ * Create the page write callback
+ *
+ * @param index
+ * @return
+ * @throws HyracksDataException
+ */
+ IPageWriteCallback createPageWriteCallback() throws HyracksDataException;
+}
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
index 7126a38..fbd74dc 100644
--- 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
@@ -31,7 +31,9 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.common.IResource;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.LocalResource;
@@ -58,6 +60,7 @@
protected final int[] filterFields;
protected final ILSMOperationTrackerFactory opTrackerProvider;
protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ protected final ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
protected final IMetadataPageManagerFactory metadataPageManagerFactory;
protected final IVirtualBufferCacheProvider vbcProvider;
protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
@@ -69,6 +72,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable) {
@@ -81,6 +85,7 @@
this.filterFields = filterFields;
this.opTrackerProvider = opTrackerProvider;
this.ioOpCallbackFactory = ioOpCallbackFactory;
+ this.pageWriteCallbackFactory = pageWriteCallbackFactory;
this.metadataPageManagerFactory = metadataPageManagerFactory;
this.vbcProvider = vbcProvider;
this.ioSchedulerProvider = ioSchedulerProvider;
@@ -131,6 +136,14 @@
filterFields = OBJECT_MAPPER.convertValue(json.get("filterFields"), int[].class);
opTrackerProvider = (ILSMOperationTrackerFactory) registry.deserialize(json.get("opTrackerProvider"));
ioOpCallbackFactory = (ILSMIOOperationCallbackFactory) registry.deserialize(json.get("ioOpCallbackFactory"));
+ if (json.has("pageWriteCallbackFactory")) {
+ // for backward-compatibility
+ pageWriteCallbackFactory =
+ (ILSMPageWriteCallbackFactory) registry.deserialize(json.get("pageWriteCallbackFactory"));
+ } else {
+ // treat legacy datasets as no op
+ pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
+ }
metadataPageManagerFactory =
(IMetadataPageManagerFactory) registry.deserialize(json.get("metadataPageManagerFactory"));
@@ -184,6 +197,7 @@
json.putPOJO("filterFields", filterFields);
json.set("opTrackerProvider", opTrackerProvider.toJson(registry));
json.set("ioOpCallbackFactory", ioOpCallbackFactory.toJson(registry));
+ json.set("pageWriteCallbackFactory", pageWriteCallbackFactory.toJson(registry));
json.set("metadataPageManagerFactory", metadataPageManagerFactory.toJson(registry));
if (vbcProvider != null) {
json.set("vbcProvider", vbcProvider.toJson(registry));
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
index c5634aa..a138f01 100644
--- 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
@@ -27,6 +27,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.common.IResourceFactory;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -42,6 +43,7 @@
protected final int[] filterFields;
protected final ILSMOperationTrackerFactory opTrackerProvider;
protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ protected final ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
protected final IMetadataPageManagerFactory metadataPageManagerFactory;
protected final IVirtualBufferCacheProvider vbcProvider;
protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
@@ -53,6 +55,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable) {
@@ -64,6 +67,7 @@
this.filterFields = filterFields;
this.opTrackerProvider = opTrackerProvider;
this.ioOpCallbackFactory = ioOpCallbackFactory;
+ this.pageWriteCallbackFactory = pageWriteCallbackFactory;
this.metadataPageManagerFactory = metadataPageManagerFactory;
this.vbcProvider = vbcProvider;
this.ioSchedulerProvider = ioSchedulerProvider;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index 1cad00f..0ad3029 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -205,41 +206,41 @@
getIndex().validate();
}
- @Override
- public IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException {
+ protected IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException {
return new FilterBulkLoader(getLSMComponentFilter(), getMetadataHolder(), getLsmIndex().getFilterManager(),
getLsmIndex().getTreeFields(), getLsmIndex().getFilterFields(),
MultiComparator.create(getLSMComponentFilter().getFilterCmpFactories()));
}
- @Override
- public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+ protected IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
return new LSMIndexBulkLoader(
- getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+ getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback));
}
/**
* Allows sub-class extend this method to use specialized bulkloader for merge
*/
protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
- return this.createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ long numElementsHint, boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
+ return createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
}
@Override
public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
- boolean cleanupEmptyComponent) throws HyracksDataException {
+ boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
if (withFilter && getLsmIndex().getFilterFields() != null) {
chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
}
IChainedComponentBulkLoader indexBulkloader = operation.getIOOpertionType() == LSMIOOperationType.MERGE
- ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex)
- : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback)
+ : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
chainedBulkLoader.addBulkLoader(indexBulkloader);
+
+ callback.initialize(chainedBulkLoader);
return chainedBulkLoader;
}
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 cf80472..01a140f 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
@@ -22,6 +22,7 @@
import java.io.IOException;
import java.nio.file.Paths;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@@ -64,12 +65,14 @@
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.ILSMPageWriteCallbackFactory;
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.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.util.trace.ITracer;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -107,21 +110,24 @@
protected final ILSMDiskComponentFactory componentFactory;
// Factory for creating on-disk index components during bulkload.
protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
+ protected final ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
private int numScheduledFlushes = 0;
public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMDiskComponentFactory componentFactory,
- ILSMDiskComponentFactory bulkLoadComponentFactory, ILSMComponentFilterFrameFactory filterFrameFactory,
- LSMComponentFilterManager filterManager, int[] filterFields, boolean durable,
- IComponentFilterHelper filterHelper, int[] treeFields, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
+ ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
+ int[] filterFields, boolean durable, IComponentFilterHelper filterHelper, int[] treeFields, ITracer tracer)
+ throws HyracksDataException {
this.ioManager = ioManager;
this.virtualBufferCaches = virtualBufferCaches;
this.diskBufferCache = diskBufferCache;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
+ this.pageWriteCallbackFactory = pageWriteCallbackFactory;
this.componentFactory = componentFactory;
this.bulkLoadComponentFactory = bulkLoadComponentFactory;
this.filterHelper = filterHelper;
@@ -148,13 +154,15 @@
public AbstractLSMIndex(IIOManager ioManager, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
- boolean durable, ITracer tracer) throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILSMDiskComponentFactory componentFactory,
+ ILSMDiskComponentFactory bulkLoadComponentFactory, boolean durable, ITracer tracer)
+ throws HyracksDataException {
this.ioManager = ioManager;
this.diskBufferCache = diskBufferCache;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
+ this.pageWriteCallbackFactory = pageWriteCallbackFactory;
this.componentFactory = componentFactory;
this.bulkLoadComponentFactory = bulkLoadComponentFactory;
this.durable = durable;
@@ -496,8 +504,8 @@
@Override
public final IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
- return createBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, null);
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
+ return createBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, Collections.emptyMap());
}
@Override
@@ -897,4 +905,8 @@
.of(IndexComponentFileReference.of(Paths.get(fileName).getFileName().toString()).getSequenceEnd());
}
+ public ILSMPageWriteCallbackFactory getPageWriteCallbackFactory() {
+ return pageWriteCallbackFactory;
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
index a9c70e0..5c22dd0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
@@ -81,4 +81,9 @@
public Throwable getFailure() {
return bulkLoader.getFailure();
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ bulkLoader.force();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
index 3fa45c9..1516136 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -158,4 +158,11 @@
}
return null;
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ for (IChainedComponentBulkLoader bulkLoader : bulkloaderChain) {
+ bulkLoader.force();
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index a2d9fdd..9a6fb43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
public class EmptyComponent implements ILSMDiskComponent {
@@ -136,20 +137,9 @@
}
@Override
- public IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException {
- return null;
- }
-
- @Override
- public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
- return null;
- }
-
- @Override
public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
- boolean cleanupEmptyComponent) throws HyracksDataException {
+ boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
index 515532e..1600d494a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
@@ -95,4 +95,9 @@
public Throwable getFailure() {
return null;
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ // no op
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
index 1361c79..e10de3b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
@@ -61,4 +61,10 @@
* @throws HyracksDataException
*/
void cleanupArtifacts() throws HyracksDataException;
+
+ /**
+ * Force all pages written by this bulkloader to disk.
+ * @throws HyracksDataException
+ */
+ void force() throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
index 394126d..17dfb1d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
@@ -90,4 +90,10 @@
}
return null;
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ bulkLoader.force();
+ buddyBTreeBulkLoader.force();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 977697b..13a0e27 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -80,4 +80,9 @@
public Throwable getFailure() {
return bulkLoader.getFailure();
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ bulkLoader.force();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
index 9b36383..c1cd130 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -39,7 +39,8 @@
this.lsmIndex = lsmIndex;
this.opCtx = opCtx;
this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
- fillFactor, verifyInput, numElementsHint, false, true, true);
+ fillFactor, verifyInput, numElementsHint, false, true, true,
+ lsmIndex.getPageWriteCallbackFactory().createPageWriteCallback());
}
public ILSMDiskComponent getComponent() {
@@ -132,4 +133,9 @@
loadOp.cleanup(lsmIndex.getBufferCache());
}
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ componentBulkLoader.force();
+ }
}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexPageWriteCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexPageWriteCallback.java
new file mode 100644
index 0000000..991968f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexPageWriteCallback.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class LSMIndexPageWriteCallback implements IPageWriteCallback {
+
+ private final int pagesPerForce;
+ private IIndexBulkLoader bulkLoader;
+ private long totalWrittenPages;
+ private int totalForces;
+
+ public LSMIndexPageWriteCallback(int pagesPerForce) {
+ this.pagesPerForce = pagesPerForce;
+ }
+
+ @Override
+ public void initialize(IIndexBulkLoader bulkLoader) {
+ this.bulkLoader = bulkLoader;
+ }
+
+ @Override
+ public void afterWrite(ICachedPage page) throws HyracksDataException {
+ totalWrittenPages++;
+ if (pagesPerForce > 0 && totalWrittenPages % pagesPerForce == 0) {
+ bulkLoader.force();
+ totalForces++;
+ }
+ }
+
+ public int getTotalForces() {
+ return totalForces;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index 9bd5007..5edcec8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -27,7 +27,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
import org.apache.hyracks.storage.common.file.IFileMapManager;
import org.apache.hyracks.util.JSONUtil;
@@ -157,13 +159,8 @@
}
@Override
- public IFIFOPageQueue createFIFOQueue() {
- throw new UnsupportedOperationException("Virtual buffer caches don't have FIFO writers");
- }
-
- @Override
- public void finishQueue() {
- throw new UnsupportedOperationException("Virtual buffer caches don't have FIFO writers");
+ public IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback) {
+ return vbc.createFIFOWriter(callback, failureCallback);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpPageWriteCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpPageWriteCallbackFactory.java
new file mode 100644
index 0000000..3d6a770
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpPageWriteCallbackFactory.java
@@ -0,0 +1,60 @@
+/*
+ * 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 org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.common.IResource;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class NoOpPageWriteCallbackFactory implements ILSMPageWriteCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final NoOpPageWriteCallbackFactory INSTANCE = new NoOpPageWriteCallbackFactory();
+
+ private NoOpPageWriteCallbackFactory() {
+ }
+
+ @Override
+ public IPageWriteCallback createPageWriteCallback() throws HyracksDataException {
+ return NoOpPageWriteCallback.INSTANCE;
+ }
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) {
+ // No op
+ }
+
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ return registry.getClassIdentifier(getClass(), serialVersionUID);
+ }
+
+ @SuppressWarnings("squid:S1172") // unused parameter
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+ return INSTANCE;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index 0a7c842..f24289f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -34,7 +34,9 @@
import org.apache.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
import org.apache.hyracks.storage.common.buffercache.VirtualPage;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
import org.apache.hyracks.storage.common.file.FileMapManager;
@@ -402,12 +404,7 @@
}
@Override
- public IFIFOPageQueue createFIFOQueue() {
- throw new UnsupportedOperationException("Virtual buffer caches don't have FIFO writers");
- }
-
- @Override
- public void finishQueue() {
+ public IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback) {
throw new UnsupportedOperationException("Virtual buffer caches don't have FIFO writers");
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 5614ca5..0904c4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -38,6 +38,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
@@ -67,6 +68,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
@@ -74,8 +76,8 @@
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
@@ -111,21 +113,22 @@
ILSMMergePolicy mergePolicy = mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx);
ILSMIOOperationScheduler ioScheduler = ioSchedulerProvider.getIoScheduler(serviceCtx);
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
if (isPartitioned) {
return InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits,
cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache,
file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
- invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
metadataPageManagerFactory, serviceCtx.getTracer());
} else {
return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache, file.getAbsolutePath(),
bloomFilterFalsePositiveRate, mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this),
- ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories,
- filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
- metadataPageManagerFactory, serviceCtx.getTracer());
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits,
+ filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, durable, metadataPageManagerFactory, serviceCtx.getTracer());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index 50fedb5..c36e8d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -28,6 +28,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -50,6 +51,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
@@ -57,8 +59,8 @@
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
@@ -73,9 +75,10 @@
public IResource createResource(FileReference fileRef) {
return new LSMInvertedIndexLocalResource(fileRef.getRelativePath(), storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
- durable, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, isPartitioned, invertedIndexFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
+ tokenizerFactory, isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 20b7a02..499720a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -53,6 +53,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
@@ -98,11 +99,13 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
+ int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
+ ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
- opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, componentFactory, filterFrameFactory,
- filterManager, filterFields, durable, filterHelper, invertedIndexFields, tracer);
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+ componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
+ invertedIndexFields, tracer);
this.tokenizerFactory = tokenizerFactory;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
@@ -292,8 +295,8 @@
btreeCountingCursor.destroy();
}
- ILSMDiskComponentBulkLoader componentBulkLoader =
- component.createBulkLoader(operation, 1.0f, false, numBTreeTuples, false, false, false);
+ ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(operation, 1.0f, false,
+ numBTreeTuples, false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
// Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -365,14 +368,15 @@
numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
.getBloomFilter().getNumElements();
}
- componentBulkLoader =
- component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false,
+ false, pageWriteCallbackFactory.createPageWriteCallback());
loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
} finally {
btreeCursor.destroy();
}
} else {
- componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false,
+ pageWriteCallbackFactory.createPageWriteCallback());
}
search(opCtx, cursor, mergePred);
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index 41e72cd..6de6ab5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
public class LSMInvertedIndexDiskComponent extends AbstractLSMWithBuddyDiskComponent {
@@ -120,11 +121,11 @@
@Override
protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+ long numElementsHint, boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
IIndexBulkLoader indexBulkLoader =
- invIndex.createMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ invIndex.createMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
IIndexBulkLoader buddyBulkLoader =
- getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, callback);
return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 604d374..23d6221 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
@@ -52,13 +53,14 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
+ int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
+ ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 4970ccb..21d1acf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -39,6 +39,7 @@
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
public class InMemoryInvertedIndex implements IInPlaceInvertedIndex {
@@ -203,7 +204,7 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
throw new UnsupportedOperationException("Bulk load not supported by in-memory inverted index.");
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 4f90e6e..2996bcf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -60,7 +60,9 @@
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -250,10 +252,10 @@
protected final boolean verifyInput;
protected final MultiComparator allCmp;
- protected final IFIFOPageQueue queue;
+ protected final IFIFOPageWriter queue;
public AbstractOnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+ boolean checkIfEmptyIndex, int startPageId, IPageWriteCallback callback) throws HyracksDataException {
this.verifyInput = verifyInput;
this.invListCmp = MultiComparator.create(invListCmpFactories);
if (verifyInput) {
@@ -265,16 +267,16 @@
this.btreeTupleReference = new ArrayTupleReference();
this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
this.lastTuple = new ArrayTupleReference();
- this.btreeBulkloader =
- btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint,
+ checkIfEmptyIndex, NoOpPageWriteCallback.INSTANCE);
currentPageId = startPageId;
currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
- queue = bufferCache.createFIFOQueue();
+ queue = bufferCache.createFIFOWriter(callback, this);
}
protected void pinNextPage() throws HyracksDataException {
- queue.put(currentPage, this);
+ queue.write(currentPage);
currentPageId++;
currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
}
@@ -354,10 +356,9 @@
btreeBulkloader.end();
if (currentPage != null) {
- queue.put(currentPage, this);
+ queue.write(currentPage);
}
invListsMaxPageId = currentPageId;
- bufferCache.finishQueue();
if (hasFailed()) {
throw HyracksDataException.create(getFailure());
}
@@ -369,13 +370,20 @@
btreeBulkloader.abort();
}
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ btreeBulkloader.force();
+ bufferCache.force(fileId, false);
+ }
+
}
public class OnDiskInvertedIndexMergeBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
public OnDiskInvertedIndexMergeBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
- super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+ boolean checkIfEmptyIndex, int startPageId, IPageWriteCallback callback) throws HyracksDataException {
+ super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId, callback);
}
@Override
@@ -402,8 +410,8 @@
public class OnDiskInvertedIndexBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
- super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+ boolean checkIfEmptyIndex, int startPageId, IPageWriteCallback callback) throws HyracksDataException {
+ super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId, callback);
}
@Override
@@ -558,15 +566,15 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
- rootPageId);
+ rootPageId, callback);
}
public IIndexBulkLoader createMergeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
return new OnDiskInvertedIndexMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
- rootPageId);
+ rootPageId, callback);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 1e812b4..c9cd959 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -41,6 +41,7 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.frames.LSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -127,10 +128,11 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
- ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
- int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- IMetadataPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+ boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer)
+ throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -166,8 +168,8 @@
return new LSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -176,10 +178,10 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
- ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
- int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+ boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -215,7 +217,7 @@
return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper,
filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
+ filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
index f74a23b..9404d91 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
@@ -35,6 +35,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
import org.apache.hyracks.storage.common.IIndex;
@@ -54,6 +55,7 @@
IBinaryComparatorFactory[] rtreeCmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
@@ -61,9 +63,9 @@
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
boolean isPointMBR, double bloomFilterFalsePositiveRate) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
+ valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
bloomFilterFalsePositiveRate);
}
@@ -81,12 +83,14 @@
IIOManager ioManager = ncServiceCtx.getIoManager();
FileReference fileRef = ioManager.resolve(path);
ioOpCallbackFactory.initialize(ncServiceCtx, this);
+ pageWriteCallbackFactory.initialize(ncServiceCtx, this);
return LSMRTreeUtils.createExternalRTree(ioManager, fileRef, storageManager.getBufferCache(ncServiceCtx),
typeTraits, cmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
bloomFilterFalsePositiveRate, mergePolicyFactory.createMergePolicy(mergePolicyProperties, ncServiceCtx),
opTrackerProvider.getOperationTracker(ncServiceCtx, this),
- ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, linearizeCmpFactory,
- buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory, ncServiceCtx.getTracer());
+ ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
+ linearizeCmpFactory, buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory,
+ ncServiceCtx.getTracer());
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
index a5a7cd8..1f4971d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
@@ -30,6 +30,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -42,6 +43,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
@@ -49,9 +51,9 @@
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
boolean isPointMBR, double bloomFilterFalsePositiveRate) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, null, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
+ valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
bloomFilterFalsePositiveRate);
}
@@ -59,9 +61,9 @@
public LsmResource createResource(FileReference fileRef) {
return new ExternalRTreeLocalResource(fileRef.getRelativePath(), storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
- metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
- btreeCmpFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields,
- buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+ pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
+ mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index d744ab3..bfae1cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -37,6 +37,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
@@ -66,6 +67,7 @@
IBinaryComparatorFactory[] rtreeCmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
@@ -73,8 +75,8 @@
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
boolean isPointMBR, double bloomFilterFalsePositiveRate) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -107,14 +109,15 @@
FileReference fileRef = ioManager.resolve(path);
List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(ncServiceCtx, fileRef);
ioOpCallbackFactory.initialize(ncServiceCtx, this);
+ pageWriteCallbackFactory.initialize(ncServiceCtx, this);
return LSMRTreeUtils.createLSMTree(ioManager, virtualBufferCaches, fileRef,
storageManager.getBufferCache(ncServiceCtx), typeTraits, cmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, bloomFilterFalsePositiveRate,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, ncServiceCtx),
opTrackerProvider.getOperationTracker(ncServiceCtx, this),
- ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, linearizeCmpFactory, rtreeFields,
- buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields, durable, isPointMBR,
- metadataPageManagerFactory);
+ ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
+ linearizeCmpFactory, rtreeFields, buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields,
+ durable, isPointMBR, metadataPageManagerFactory);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
index 8d27caa..635f1ac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
@@ -30,6 +30,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -53,6 +54,7 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
@@ -60,8 +62,8 @@
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
boolean isPointMBR, double bloomFilterFalsePositiveRate) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -76,8 +78,9 @@
public IResource createResource(FileReference fileRef) {
return new LSMRTreeLocalResource(fileRef.getRelativePath(), storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
- metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
- durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields,
- buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
+ rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
+ bloomFilterFalsePositiveRate);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index 4b6f1de..e4a8287 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -38,6 +38,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
@@ -64,14 +65,15 @@
IBinaryComparatorFactory[] rtreeCmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -99,13 +101,14 @@
FileReference file = ioManager.resolve(path);
List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, file,
storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
- ioOpCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits, filterCmpFactories,
- filterFields, durable, isPointMBR, metadataPageManagerFactory);
+ ioOpCallbackFactory, pageWriteCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits,
+ filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
index e633c2a..6edf877 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
@@ -30,6 +30,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
@@ -50,6 +51,7 @@
IBinaryComparatorFactory[] rtreeCmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable,
@@ -57,8 +59,8 @@
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR,
IBinaryComparatorFactory[] btreeComparatorFactories) {
super(storageManager, rteeTypeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable);
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
this.linearizeCmpFactory = linearizeCmpFactory;
@@ -71,8 +73,8 @@
public LsmResource createResource(FileReference fileRef) {
return new LSMRTreeWithAntiMatterLocalResource(fileRef.getRelativePath(), storageManager, typeTraits,
cmpFactories, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
- ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
- mergePolicyProperties, btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
- linearizeCmpFactory, rtreeFields, isPointMBR, durable);
+ ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider,
+ ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, btreeComparatorFactories,
+ valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index c695a5c..b04570e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -45,6 +45,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
@@ -83,12 +84,13 @@
ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- IComponentFilterHelper filterHelper, ILSMComponentFilterFrameFactory filterFrameFactory,
- LSMComponentFilterManager filterManager, int[] rtreeFields, int[] filterFields, boolean durable,
- boolean isPointMBR) throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, IComponentFilterHelper filterHelper,
+ ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
+ int[] rtreeFields, int[] filterFields, boolean durable, boolean isPointMBR) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
- opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, bulkLoadComponentFactory,
- filterFrameFactory, filterManager, filterFields, durable, filterHelper, rtreeFields, ITracer.NONE);
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+ bulkLoadComponentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
+ rtreeFields, ITracer.NONE);
int i = 0;
for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
RTree memRTree = new RTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
@@ -125,10 +127,12 @@
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean durable,
- boolean isPointMBR, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean durable, boolean isPointMBR, ITracer tracer)
+ throws HyracksDataException {
super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, componentFactory, componentFactory, durable, tracer);
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory, componentFactory, durable,
+ tracer);
this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
index 015f34c..24c2dbe 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -38,6 +38,7 @@
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;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -47,8 +48,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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
@@ -60,6 +61,7 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.util.trace.ITracer;
/**
@@ -86,12 +88,13 @@
IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] buddyBTreeFields, boolean durable,
- boolean isPointMBR, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] buddyBTreeFields, boolean durable, boolean isPointMBR, ITracer tracer) throws HyracksDataException {
super(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bloomFilterFalsePositiveRate,
rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
- opTracker, ioScheduler, ioOpCallbackFactory, buddyBTreeFields, durable, isPointMBR, tracer);
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, buddyBTreeFields, durable,
+ isPointMBR, tracer);
this.secondDiskComponents = new LinkedList<>();
this.fieldCount = fieldCount;
}
@@ -268,6 +271,7 @@
.get(mergeOp.getMergingComponents().size() - 1) != secondDiskComponents
.get(secondDiskComponents.size() - 1);
}
+ IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
if (keepDeleteTuples) {
// Keep the deleted tuples since the oldest disk component is not
// included in the merge operation
@@ -276,7 +280,7 @@
search(opCtx, btreeCursor, rtreeSearchPred);
BTree btree = mergedComponent.getBuddyIndex();
- IIndexBulkLoader btreeBulkLoader = btree.createBulkLoader(1.0f, true, 0L, false);
+ IIndexBulkLoader btreeBulkLoader = btree.createBulkLoader(1.0f, true, 0L, false, pageWriteCallback);
long numElements = 0L;
for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
@@ -288,7 +292,7 @@
BloomFilterSpecification bloomFilterSpec =
BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
- bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+ bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements(), pageWriteCallback);
try {
while (btreeCursor.hasNext()) {
@@ -304,7 +308,8 @@
btreeBulkLoader.end();
}
- IIndexBulkLoader bulkLoader = mergedComponent.getIndex().createBulkLoader(1.0f, false, 0L, false);
+ IIndexBulkLoader bulkLoader =
+ mergedComponent.getIndex().createBulkLoader(1.0f, false, 0L, false, pageWriteCallback);
try {
while (cursor.hasNext()) {
cursor.next();
@@ -454,7 +459,7 @@
private final ILSMDiskComponent component;
private final boolean isTransaction;
private final LoadOperation loadOp;
- private final ChainedLSMDiskComponentBulkLoader componentBulkLoader;
+ private final ILSMDiskComponentBulkLoader componentBulkLoader;
public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
@@ -482,8 +487,8 @@
loadOp.setNewComponent(component);
ioOpCallback.scheduled(loadOp);
ioOpCallback.beforeOperation(loadOp);
- componentBulkLoader =
- component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
+ componentBulkLoader = component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false,
+ true, false, pageWriteCallbackFactory.createPageWriteCallback());
}
@Override
@@ -548,6 +553,11 @@
public Throwable getFailure() {
return loadOp.getFailure();
}
+
+ @Override
+ public void force() throws HyracksDataException {
+ componentBulkLoader.force();
+ }
}
// The only change the the schedule merge is the method used to create the
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 4566d32..05182b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -53,6 +53,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
@@ -80,13 +81,15 @@
IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] rtreeFields, int[] buddyBTreeFields,
- int[] filterFields, boolean durable, boolean isPointMBR) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] rtreeFields, int[] buddyBTreeFields, int[] filterFields, boolean durable, boolean isPointMBR)
+ throws HyracksDataException {
super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
componentFactory, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
- filterHelper, filterFrameFactory, filterManager, rtreeFields, filterFields, durable, isPointMBR);
+ pageWriteCallbackFactory, filterHelper, filterFrameFactory, filterManager, rtreeFields, filterFields,
+ durable, isPointMBR);
this.buddyBTreeFields = buddyBTreeFields;
}
@@ -101,12 +104,13 @@
IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] buddyBTreeFields, boolean durable,
- boolean isPointMBR, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] buddyBTreeFields, boolean durable, boolean isPointMBR, ITracer tracer) throws HyracksDataException {
super(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, rtreeCmpFactories,
btreeCmpFactories, linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, durable, isPointMBR, tracer);
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, durable, isPointMBR,
+ tracer);
this.buddyBTreeFields = buddyBTreeFields;
}
@@ -135,7 +139,7 @@
component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
flushOp.getBloomFilterTarget(), true);
componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
- false, false, false);
+ false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
// scan the memory BTree and bulk load delete tuples
flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -331,13 +335,13 @@
numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
.getNumElements();
}
- componentBulkLoader =
- mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false, false);
+ componentBulkLoader = mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false,
+ false, false, pageWriteCallbackFactory.createPageWriteCallback());
mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
} else {
//no buddy-btree needed
- componentBulkLoader =
- mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false);
+ componentBulkLoader = mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false,
+ false, pageWriteCallbackFactory.createPageWriteCallback());
}
//search old rtree components
while (cursor.hasNext()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 2757595..20cd927 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -48,6 +48,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
@@ -77,13 +78,14 @@
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeComparatorFactories,
ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] rtreeFields, int[] filterFields, boolean durable,
- boolean isPointMBR) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] rtreeFields, int[] filterFields, boolean durable, boolean isPointMBR) throws HyracksDataException {
super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileManager, componentFactory,
bulkLoadComponentFactory, fieldCount, rtreeCmpFactories, btreeComparatorFactories, linearizer,
comparatorFields, linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
- filterHelper, filterFrameFactory, filterManager, rtreeFields, filterFields, durable, isPointMBR);
+ pageWriteCallbackFactory, filterHelper, filterFrameFactory, filterManager, rtreeFields, filterFields,
+ durable, isPointMBR);
}
@Override
@@ -109,7 +111,8 @@
try {
memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
- componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false,
+ pageWriteCallbackFactory.createPageWriteCallback());
// Since the LSM-RTree is used as a secondary assumption, the
// primary key will be the last comparator in the BTree comparators
rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
@@ -234,8 +237,8 @@
// Bulk load the tuples from all on-disk RTrees into the new RTree.
ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
- ILSMDiskComponentBulkLoader componentBulkLoader =
- component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
+ ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false,
+ false, false, pageWriteCallbackFactory.createPageWriteCallback());
try {
while (cursor.hasNext()) {
cursor.next();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 4b851e4..ba6015d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -45,6 +45,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.frames.LSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -81,10 +82,10 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
- ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
- boolean durable, boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory)
- throws HyracksDataException {
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
+ int[] rtreeFields, int[] buddyBTreeFields, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
+ IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
int valueFieldCount = buddyBTreeFields.length;
int keyFieldCount = typeTraits.length - valueFieldCount;
ITypeTraits[] btreeTypeTraits = new ITypeTraits[valueFieldCount];
@@ -140,8 +141,8 @@
btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, rtreeFields, buddyBTreeFields, filterFields,
- durable, isPointMBR);
+ mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, rtreeFields,
+ buddyBTreeFields, filterFields, durable, isPointMBR);
}
public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IIOManager ioManager,
@@ -150,10 +151,10 @@
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
- int[] filterFields, boolean durable, boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory)
- throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ ILinearizeComparatorFactory linearizerCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
+ IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
new LSMRTreeTupleWriterFactory(typeTraits, false);
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory;
@@ -234,7 +235,7 @@
fileNameManager, componentFactory, bulkLoadComponentFactory, filterHelper, filterFrameFactory,
filterManager, typeTraits.length, rtreeCmpFactories, btreeComparatorFactories, linearizerCmpFactory,
comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
- rtreeFields, filterFields, durable, isPointMBR);
+ pageWriteCallbackFactory, rtreeFields, filterFields, durable, isPointMBR);
}
public static ExternalRTree createExternalRTree(IIOManager ioManager, FileReference file,
@@ -242,9 +243,10 @@
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
- int[] buddyBTreeFields, boolean durable, boolean isPointMBR,
- IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer) throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ ILinearizeComparatorFactory linearizeCmpFactory, int[] buddyBTreeFields, boolean durable,
+ boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
+ throws HyracksDataException {
int keyFieldCount = rtreeCmpFactories.length;
int valueFieldCount = typeTraits.length - keyFieldCount;
@@ -290,8 +292,8 @@
return new ExternalRTree(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bloomFilterFalsePositiveRate,
typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields,
- linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, buddyBTreeFields, durable,
- isPointMBR, tracer);
+ linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
+ buddyBTreeFields, durable, isPointMBR, tracer);
}
public static ILinearizeComparatorFactory proposeBestLinearizer(ITypeTraits[] typeTraits, int numKeyFields)
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index b34a13c..f31049a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -60,6 +60,7 @@
import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public class RTree extends AbstractTreeIndex {
@@ -892,9 +893,9 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException {
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
// TODO: verifyInput currently does nothing.
- return new RTreeBulkLoader(fillFactor);
+ return new RTreeBulkLoader(fillFactor, callback);
}
public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
@@ -905,8 +906,8 @@
ByteBuffer mbr;
List<Integer> prevNodeFrontierPages = new ArrayList<>();
- public RTreeBulkLoader(float fillFactor) throws HyracksDataException {
- super(fillFactor);
+ public RTreeBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
+ super(fillFactor, callback);
prevInteriorFrame = interiorFrameFactory.createFrame();
}
@@ -942,9 +943,9 @@
leafFrontier.pageId = freePageManager.takePage(metaFrame);
- putInQueue(leafFrontier.page);
+ write(leafFrontier.page);
for (ICachedPage c : pagesToWrite) {
- putInQueue(c);
+ write(c);
}
pagesToWrite.clear();
leafFrontier.page = bufferCache
@@ -975,7 +976,7 @@
}
for (ICachedPage c : pagesToWrite) {
- putInQueue(c);
+ write(c);
}
finish();
super.end();
@@ -1012,7 +1013,7 @@
((RTreeNSMFrame) lowerFrame).adjustMBR();
interiorFrameTupleWriter.writeTupleFields(((RTreeNSMFrame) lowerFrame).getMBRTuples(), 0, mbr, 0);
}
- putInQueue(n.page);
+ write(n.page);
n.page = null;
prevPageId = n.pageId;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index 31037ff..0384dca 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -45,6 +45,11 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-dataflow-common</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
index 4c0012d..861fd2d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
/**
* This interface describes the operations common to all indexes. Indexes
@@ -137,7 +138,7 @@
* @throws HyracksDataException
*/
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex) throws HyracksDataException;
+ boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException;
/**
* TODO: This should be moved to ILSMIndex since filters don't make sense in non LSM context
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index 138705f..bae3931 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -52,4 +52,10 @@
* If the operation was completed through end() invocation before abort is called
*/
void abort() throws HyracksDataException;
+
+ /**
+ * Force all pages written by this bulkloader to disk.
+ * @throws HyracksDataException
+ */
+ void force() throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
index a060972..44a2031 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AbstractBufferedFileIOManager.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.io.IFileHandle;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.util.IoUtil;
+import org.apache.hyracks.control.nc.io.IOManager;
import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.util.annotations.NotThreadSafe;
@@ -42,7 +43,7 @@
protected final BufferCache bufferCache;
protected final IPageReplacementStrategy pageReplacementStrategy;
private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache;
- private final IIOManager ioManager;
+ private final IOManager ioManager;
private IFileHandle fileHandle;
private volatile boolean hasOpen;
@@ -50,7 +51,7 @@
protected AbstractBufferedFileIOManager(BufferCache bufferCache, IIOManager ioManager,
BlockingQueue<BufferCacheHeaderHelper> headerPageCache, IPageReplacementStrategy pageReplacementStrategy) {
this.bufferCache = bufferCache;
- this.ioManager = ioManager;
+ this.ioManager = (IOManager) ioManager;
this.headerPageCache = headerPageCache;
this.pageReplacementStrategy = pageReplacementStrategy;
hasOpen = false;
@@ -276,11 +277,11 @@
}
protected final long writeToFile(ByteBuffer buf, long offset) throws HyracksDataException {
- return ioManager.syncWrite(fileHandle, offset, buf);
+ return ioManager.doSyncWrite(fileHandle, offset, buf);
}
protected final long writeToFile(ByteBuffer[] buf, long offset) throws HyracksDataException {
- return ioManager.syncWrite(fileHandle, offset, buf);
+ return ioManager.doSyncWrite(fileHandle, offset, buf);
}
protected final long getFileSize() {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
deleted file mode 100644
index 589d697..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
+++ /dev/null
@@ -1,185 +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.common.buffercache;
-
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.ExitUtil;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class AsyncFIFOPageQueueManager implements Runnable {
- private static final boolean DEBUG = false;
- private static final Logger LOGGER = LogManager.getLogger();
-
- protected LinkedBlockingQueue<ICachedPage> queue = new LinkedBlockingQueue<>();
- volatile Thread writerThread;
- protected AtomicBoolean poisoned = new AtomicBoolean(false);
- protected BufferCache bufferCache;
- protected volatile PageQueue pageQueue;
-
- public AsyncFIFOPageQueueManager(BufferCache bufferCache) {
- this.bufferCache = bufferCache;
- }
-
- protected class PageQueue implements IFIFOPageQueue {
- final IBufferCache bufferCache;
- public final IFIFOPageWriter writer;
-
- protected PageQueue(IBufferCache bufferCache, IFIFOPageWriter writer) {
- if (DEBUG)
- System.out.println("[FIFO] New Queue");
- this.bufferCache = bufferCache;
- this.writer = writer;
- }
-
- protected IBufferCache getBufferCache() {
- return bufferCache;
- }
-
- protected IFIFOPageWriter getWriter() {
- return writer;
- }
-
- @SuppressWarnings("squid:S2142")
- @Override
- public void put(ICachedPage page, IPageWriteFailureCallback callback) throws HyracksDataException {
- failIfPreviousPageFailed(callback);
- page.setFailureCallback(callback);
- try {
- if (!poisoned.get()) {
- queue.put(page);
- } else {
- LOGGER.error("An attempt to write a page found buffer cache closed");
- ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
- }
- } catch (InterruptedException e) {
- LOGGER.error("IO Operation interrupted", e);
- ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
- }
- }
-
- private void failIfPreviousPageFailed(IPageWriteFailureCallback callback) throws HyracksDataException {
- if (callback.hasFailed()) {
- throw HyracksDataException.create(callback.getFailure());
- }
- }
- }
-
- public PageQueue createQueue(IFIFOPageWriter writer) {
- if (pageQueue == null) {
- synchronized (this) {
- if (pageQueue == null) {
- writerThread = new Thread(this);
- writerThread.setName("FIFO Writer Thread");
- writerThread.start();
- pageQueue = new PageQueue(bufferCache, writer);
- }
- }
- }
- return pageQueue;
- }
-
- public void destroyQueue() {
- poisoned.set(true);
- if (writerThread == null) {
- synchronized (this) {
- if (writerThread == null) {
- return;
- }
- }
- }
-
- //Dummy cached page to act as poison pill
- CachedPage poisonPill = new CachedPage();
- poisonPill.setQueueInfo(new QueueInfo(true, true));
-
- try {
- synchronized (poisonPill) {
- queue.put(poisonPill);
- while (queue.contains(poisonPill)) {
- poisonPill.wait();
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
-
- public void finishQueue() throws HyracksDataException {
- if (writerThread == null) {
- synchronized (this) {
- if (writerThread == null) {
- return;
- }
- }
- }
- try {
- //Dummy cached page to act as low water mark
- CachedPage lowWater = new CachedPage();
- lowWater.setQueueInfo(new QueueInfo(true, false));
- synchronized (lowWater) {
- queue.put(lowWater);
- while (queue.contains(lowWater)) {
- lowWater.wait();
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw HyracksDataException.create(e);
- }
- }
-
- @SuppressWarnings("squid:S2142")
- @Override
- public void run() {
- if (DEBUG) {
- LOGGER.info("[FIFO] Writer started");
- }
- boolean die = false;
- while (!die) {
- ICachedPage entry;
- try {
- entry = queue.take();
- } catch (InterruptedException e) {
- LOGGER.error("BufferCache Write Queue was interrupted", e);
- ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
- return; // Keep compiler happy
- }
- if (entry.getQueueInfo() != null && entry.getQueueInfo().hasWaiters()) {
- synchronized (entry) {
- if (entry.getQueueInfo().isPoison()) {
- die = true;
- }
- entry.notifyAll();
- continue;
- }
- }
- if (DEBUG) {
- LOGGER.info("[FIFO] Write " + BufferedFileHandle.getFileId(((CachedPage) entry).dpid) + ","
- + BufferedFileHandle.getPageId(((CachedPage) entry).dpid));
- }
- pageQueue.getWriter().write(entry, bufferCache);
- }
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 55875a6..3c08462 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -75,7 +75,6 @@
private final IFileMapManager fileMapManager;
private final CleanerThread cleanerThread;
private final Map<Integer, BufferedFileHandle> fileInfoMap;
- private final AsyncFIFOPageQueueManager fifoWriter;
private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache;
private IIOReplicationManager ioReplicationManager;
@@ -114,7 +113,6 @@
executor.execute(cleanerThread);
closed = false;
- fifoWriter = new AsyncFIFOPageQueueManager(this);
if (DEBUG) {
confiscatedPages = new ArrayList<>();
confiscatedPagesOwner = new HashMap<>();
@@ -716,7 +714,6 @@
@Override
public void close() {
closed = true;
- fifoWriter.destroyQueue();
try {
synchronized (cleanerThread.threadLock) {
cleanerThread.shutdownStart = true;
@@ -1302,7 +1299,6 @@
Thread.currentThread().interrupt();
}
}
- finishQueue();
if (cycleCount > MAX_PIN_ATTEMPT_CYCLES) {
cycleCount = 0; // suppress warning below
throw new HyracksDataException("Unable to find free page in buffer cache after "
@@ -1378,13 +1374,8 @@
}
@Override
- public IFIFOPageQueue createFIFOQueue() {
- return fifoWriter.createQueue(FIFOLocalWriter.INSTANCE);
- }
-
- @Override
- public void finishQueue() throws HyracksDataException {
- fifoWriter.finishQueue();
+ public IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback) {
+ return new FIFOLocalWriter(this, callback, failureCallback);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
index f66bc98..1d4c789 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
@@ -23,14 +23,10 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
/**
* @author yingyib
*/
public class CachedPage implements ICachedPageInternal {
- private static final Logger LOGGER = LogManager.getLogger();
final int cpid;
ByteBuffer buffer;
public final AtomicInteger pinCount;
@@ -42,7 +38,6 @@
CachedPage next;
volatile boolean valid;
final AtomicBoolean confiscated;
- private IQueueInfo queueInfo;
private int multiplier;
private int extraBlockPageId;
private long compressedOffset;
@@ -50,7 +45,6 @@
// DEBUG
private static final boolean DEBUG = false;
private final StackTraceElement[] ctorStack;
- private IPageWriteFailureCallback failureCallback;
//Constructor for making dummy entry for FIFO queue
public CachedPage() {
@@ -60,7 +54,6 @@
this.dirty = new AtomicBoolean(false);
this.confiscated = new AtomicBoolean(true);
pinCount = null;
- queueInfo = null;
replacementStrategyObject = null;
latch = null;
ctorStack = DEBUG ? new Throwable().getStackTrace() : null;
@@ -81,7 +74,6 @@
dpid = -1;
valid = false;
confiscated = new AtomicBoolean(false);
- queueInfo = null;
ctorStack = DEBUG ? new Throwable().getStackTrace() : null;
}
@@ -91,8 +83,6 @@
valid = false;
confiscated.set(false);
pageReplacementStrategy.notifyCachePageReset(this);
- queueInfo = null;
- failureCallback = null;
}
public void invalidate() {
@@ -153,16 +143,6 @@
}
@Override
- public IQueueInfo getQueueInfo() {
- return queueInfo;
- }
-
- @Override
- public void setQueueInfo(IQueueInfo queueInfo) {
- this.queueInfo = queueInfo;
- }
-
- @Override
public long getDiskPageId() {
return dpid;
}
@@ -210,23 +190,6 @@
return multiplier > 1;
}
- @Override
- public void setFailureCallback(IPageWriteFailureCallback failureCallback) {
- if (this.failureCallback != null) {
- throw new IllegalStateException("failureCallback is already set");
- }
- this.failureCallback = failureCallback;
- }
-
- @Override
- public void writeFailed(Exception e) {
- if (failureCallback != null) {
- failureCallback.writeFailed(this, e);
- } else {
- LOGGER.error("An IO Failure took place but the failure callback is not set", e);
- }
- }
-
public void setCompressedPageOffset(long offset) {
this.compressedOffset = offset;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 9e92f21..8c3d492 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -194,13 +194,8 @@
}
@Override
- public IFIFOPageQueue createFIFOQueue() {
- return bufferCache.createFIFOQueue();
- }
-
- @Override
- public void finishQueue() throws HyracksDataException {
- bufferCache.finishQueue();
+ public IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback) {
+ return bufferCache.createFIFOWriter(callback, failureCallback);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
index 3d3ce3c..5f8bb4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
@@ -21,20 +21,29 @@
public class FIFOLocalWriter implements IFIFOPageWriter {
private static final Logger LOGGER = LogManager.getLogger();
- public static final FIFOLocalWriter INSTANCE = new FIFOLocalWriter();
private static final boolean DEBUG = false;
- private FIFOLocalWriter() {
+ private final BufferCache bufferCache;
+
+ private final IPageWriteCallback callback;
+ private final IPageWriteFailureCallback failureCallback;
+
+ public FIFOLocalWriter(BufferCache bufferCache, IPageWriteCallback callback,
+ IPageWriteFailureCallback failureCallback) {
+ this.bufferCache = bufferCache;
+ this.callback = callback;
+ this.failureCallback = failureCallback;
}
@SuppressWarnings("squid:S1181") // System must halt on all IO errors
@Override
- public void write(ICachedPage page, BufferCache bufferCache) {
+ public void write(ICachedPage page) {
CachedPage cPage = (CachedPage) page;
try {
bufferCache.write(cPage);
+ callback.afterWrite(cPage);
} catch (Exception e) {
- page.writeFailed(e);
+ handleWriteFailure(page, e);
LOGGER.warn("Failed to write page {}", cPage, e);
} catch (Throwable th) {
// Halt
@@ -48,4 +57,12 @@
}
}
+ private void handleWriteFailure(ICachedPage page, Exception e) {
+ if (failureCallback != null) {
+ failureCallback.writeFailed(page, e);
+ } else {
+ LOGGER.error("an IO failure took place but the failure callback is not set", e);
+ }
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index c762dd9..df0bea8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -229,14 +229,9 @@
void close() throws HyracksDataException;
/**
- * @return an instance of {@link IFIFOPageQueue} that can be used to write pages to the file
+ * @return an instance of {@link IFIFOPageWriter} that can be used to write pages to the file
*/
- IFIFOPageQueue createFIFOQueue();
-
- /**
- * Flush the queued pages written through buffer cache FIFO queues
- */
- void finishQueue() throws HyracksDataException;
+ IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback);
// TODO: remove the replication out of the buffer cache interface
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
index cfbb145..c6d06b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
@@ -34,24 +34,16 @@
boolean confiscated();
- IQueueInfo getQueueInfo();
-
- void setQueueInfo(IQueueInfo queueInfo);
-
int getPageSize();
int getFrameSizeMultiplier();
void setDiskPageId(long dpid);
- void setFailureCallback(IPageWriteFailureCallback callback);
-
/**
* Check if a page is a large page
*
* @return true if the page is large, false otherwise
*/
boolean isLargePage();
-
- void writeFailed(Exception e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java
deleted file mode 100644
index 189c402..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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.common.buffercache;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-@FunctionalInterface
-public interface IFIFOPageQueue {
-
- /**
- * Put a page in the write queue
- *
- * @param page
- * the page to be written
- * @param callback
- * callback in case of a failure
- * @throws HyracksDataException
- * if the callback has already failed. This indicates a failure writing a previous page
- * in the same operation.
- * Note: having this failure at this place removes the need to check for failures with
- * every add() call in the bulk loader and so, we check per page given to disk rather
- * than per tuple given to loader. At the same time, it allows the bulk load to fail early.
- */
- void put(ICachedPage page, IPageWriteFailureCallback callback) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
index 26fd414..31e0e85 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
@@ -17,5 +17,5 @@
@FunctionalInterface
public interface IFIFOPageWriter {
- void write(ICachedPage page, BufferCache bufferCache);
+ void write(ICachedPage page);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteCallback.java
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteCallback.java
index d86319d..3d2bda9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteCallback.java
@@ -18,24 +18,23 @@
*/
package org.apache.hyracks.storage.common.buffercache;
-public class QueueInfo implements IQueueInfo {
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
- private final boolean poison;
- private final boolean waiters;
+public interface IPageWriteCallback {
+ /**
+ * Initialization
+ *
+ * @param bulkLoader
+ */
+ void initialize(IIndexBulkLoader bulkLoader);
- public QueueInfo(boolean waiters, boolean poison) {
- this.waiters = waiters;
- this.poison = poison;
- }
+ /**
+ * Notify that a page has been written
+ *
+ * @param page
+ * @throws HyracksDataException
+ */
+ void afterWrite(ICachedPage page) throws HyracksDataException;
- @Override
- public boolean hasWaiters() {
- return waiters;
-
- }
-
- @Override
- public boolean isPoison() {
- return poison;
- }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IQueueInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IQueueInfo.java
deleted file mode 100644
index f214639..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IQueueInfo.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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.common.buffercache;
-
-public interface IQueueInfo {
-
- /**
- * Returns whether this queued page should notify all waiters (i.e. it is a low water mark)
- * @return
- */
-
- boolean hasWaiters();
-
- /**
- * Whether or not this cached page is a poisoned page.
- * @return
- */
-
- boolean isPoison();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/NoOpPageWriteCallback.java
similarity index 66%
rename from hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java
rename to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/NoOpPageWriteCallback.java
index d86319d..9fe3f16 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/QueueInfo.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/NoOpPageWriteCallback.java
@@ -18,24 +18,25 @@
*/
package org.apache.hyracks.storage.common.buffercache;
-public class QueueInfo implements IQueueInfo {
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
- private final boolean poison;
- private final boolean waiters;
+public class NoOpPageWriteCallback implements IPageWriteCallback {
+ public static final NoOpPageWriteCallback INSTANCE = new NoOpPageWriteCallback();
- public QueueInfo(boolean waiters, boolean poison) {
- this.waiters = waiters;
- this.poison = poison;
+ private NoOpPageWriteCallback() {
}
@Override
- public boolean hasWaiters() {
- return waiters;
-
+ public void initialize(IIndexBulkLoader bulkLoader) {
+ // no op
}
- @Override
- public boolean isPoison() {
- return poison;
+ public void beforeWrite(ICachedPage page) {
+ // no op
}
+
+ public void afterWrite(ICachedPage page) {
+ // no op
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
index b2a1ff2..fba993c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
@@ -73,16 +73,6 @@
}
@Override
- public IQueueInfo getQueueInfo() {
- return null;
- }
-
- @Override
- public void setQueueInfo(IQueueInfo queueInfo) {
- throw new UnsupportedOperationException();
- }
-
- @Override
public int getPageSize() {
return pageSize;
}
@@ -147,14 +137,4 @@
str.append("}");
return str.toString();
}
-
- @Override
- public void setFailureCallback(IPageWriteFailureCallback callback) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void writeFailed(Exception e) {
- throw new UnsupportedOperationException();
- }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
index dcccc52..9217415 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/LAFWriter.java
@@ -32,28 +32,25 @@
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.ICachedPageInternal;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
import org.apache.hyracks.util.annotations.NotThreadSafe;
/**
* Look Aside File writer
- * This class is called by two threads simultaneously:
- * - a thread to prepare the LAF page (bulk-loader)
- * - and a writer thread to write the LAF page (buffer cache writer thread)
- * Hence, it is not thread safe to have more than one thread to prepare or to write LAF pages.
*/
@NotThreadSafe
class LAFWriter implements ICompressedPageWriter {
private final CompressedFileManager compressedFileManager;
private final IBufferCache bufferCache;
- private final IFIFOPageQueue queue;
+ private final IFIFOPageWriter pageWriter;
private final Map<Integer, LAFFrame> cachedFrames;
private final Queue<LAFFrame> recycledFrames;
private final int fileId;
private final int maxNumOfEntries;
- private final PageWriteFailureCallback callBack;
+ private final PageWriteFailureCallback failureCallback;
private LAFFrame currentFrame;
private int currentPageId;
private int maxPageId;
@@ -64,12 +61,11 @@
public LAFWriter(CompressedFileManager compressedFileManager, IBufferCache bufferCache) {
this.compressedFileManager = compressedFileManager;
this.bufferCache = bufferCache;
- queue = bufferCache.createFIFOQueue();
cachedFrames = new HashMap<>();
recycledFrames = new ArrayDeque<>();
this.fileId = compressedFileManager.getFileId();
- callBack = new PageWriteFailureCallback();
-
+ failureCallback = new PageWriteFailureCallback();
+ pageWriter = bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, failureCallback);
maxNumOfEntries = bufferCache.getPageSize() / ENTRY_LENGTH;
lastOffset = 0;
totalNumOfPages = 0;
@@ -79,7 +75,7 @@
/* ************************************
* ICompressedPageWriter methods
- * Called by non-BufferCache thread (Bulk-loader)
+ * Called by non-BufferCache (Bulk-loader)
* ************************************
*/
@@ -88,15 +84,13 @@
final ICachedPageInternal internalPage = (ICachedPageInternal) cPage;
final int entryPageId = getLAFEntryPageId(BufferedFileHandle.getPageId(internalPage.getDiskPageId()));
- synchronized (cachedFrames) {
- if (!cachedFrames.containsKey(entryPageId)) {
- try {
- //Writing new page(s). Confiscate the page(s) from the buffer cache.
- prepareFrames(entryPageId, internalPage);
- } catch (HyracksDataException e) {
- abort();
- throw e;
- }
+ if (!cachedFrames.containsKey(entryPageId)) {
+ try {
+ //Writing new page(s). Confiscate the page(s) from the buffer cache.
+ prepareFrames(entryPageId, internalPage);
+ } catch (HyracksDataException e) {
+ abort();
+ throw e;
}
}
}
@@ -131,43 +125,38 @@
@Override
public void endWriting() throws HyracksDataException {
- if (callBack.hasFailed()) {
+ if (failureCallback.hasFailed()) {
//if write failed, return confiscated pages
abort();
- throw HyracksDataException.create(callBack.getFailure());
+ throw HyracksDataException.create(failureCallback.getFailure());
}
- synchronized (cachedFrames) {
- final LAFFrame lastPage = cachedFrames.get(maxPageId);
- if (lastPage != null && !lastPage.isFull()) {
- /*
- * The last page may or may not be filled. In case it is not filled (i.e do not have
- * the max number of entries). Then, write an indicator after the last entry.
- * If it has been written (i.e lastPage = null), that means it has been filled.
- */
- lastPage.setEOF();
- }
- for (Entry<Integer, LAFFrame> entry : cachedFrames.entrySet()) {
- queue.put(entry.getValue().cPage, callBack);
- }
- bufferCache.finishQueue();
+ final LAFFrame lastPage = cachedFrames.get(maxPageId);
+ if (lastPage != null && !lastPage.isFull()) {
+ /*
+ * The last page may or may not be filled. In case it is not filled (i.e do not have
+ * the max number of entries). Then, write an indicator after the last entry.
+ * If it has been written (i.e lastPage = null), that means it has been filled.
+ */
+ lastPage.setEOF();
+ }
+ for (Entry<Integer, LAFFrame> entry : cachedFrames.entrySet()) {
+ pageWriter.write(entry.getValue().cPage);
+ }
- //Signal the compressedFileManager to change its state
- compressedFileManager.endWriting(totalNumOfPages);
- }
+ //Signal the compressedFileManager to change its state
+ compressedFileManager.endWriting(totalNumOfPages);
}
@Override
public void abort() {
- synchronized (cachedFrames) {
- for (Entry<Integer, LAFFrame> frame : cachedFrames.entrySet()) {
- bufferCache.returnPage(frame.getValue().cPage);
- }
+ for (Entry<Integer, LAFFrame> frame : cachedFrames.entrySet()) {
+ bufferCache.returnPage(frame.getValue().cPage);
}
}
/* ************************************
* Local methods:
- * Called by BufferCache writer thread
+ * Called by BufferCache writer
* ************************************
*/
@@ -191,14 +180,12 @@
}
final LAFFrame frame;
- synchronized (cachedFrames) {
- //Check if the frame is cached
- frame = cachedFrames.get(pageId);
- if (frame == null) {
- //Trying to write unprepared page
- abort();
- throw new IllegalStateException("Unprepared compressed-write for page ID: " + pageId);
- }
+ //Check if the frame is cached
+ frame = cachedFrames.get(pageId);
+ if (frame == null) {
+ //Trying to write unprepared page
+ abort();
+ throw new IllegalStateException("Unprepared compressed-write for page ID: " + pageId);
}
currentFrame = frame;
@@ -210,13 +197,11 @@
if (currentFrame.isFull()) {
//The LAF page is filled. We do not need to keep it.
//Write it to the file and remove it from the cachedFrames map
- queue.put(currentFrame.cPage, callBack);
- synchronized (cachedFrames) {
- //Recycle the frame
- final LAFFrame frame = cachedFrames.remove(currentPageId);
- frame.setCachedPage(null);
- recycledFrames.add(frame);
- }
+ pageWriter.write(currentFrame.cPage);
+ //Recycle the frame
+ final LAFFrame frame = cachedFrames.remove(currentPageId);
+ frame.setCachedPage(null);
+ recycledFrames.add(frame);
currentFrame = null;
currentPageId = -1;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 7b20ea1..c37d781 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -49,6 +49,7 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.Test;
@@ -646,7 +647,8 @@
LOGGER.info("Bulk loading " + ins + " tuples");
}
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins, true);
+ IIndexBulkLoader bulkLoader =
+ treeIndex.createBulkLoader(0.7f, false, ins, true, NoOpPageWriteCallback.INSTANCE);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
for (int i = 0; i < ins; i++) {
@@ -719,7 +721,8 @@
treeIndex.activate();
// Load sorted records, and expect to fail at tuple i.
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins, true);
+ IIndexBulkLoader bulkLoader =
+ treeIndex.createBulkLoader(0.7f, true, ins, true, NoOpPageWriteCallback.INSTANCE);
for (int j = 0; j < ins; j++) {
if (j > i) {
fail("Bulk load failure test unexpectedly succeeded past tuple: " + j);
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
index a703b0b..43d1504 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -334,7 +335,8 @@
filtered ? new ArrayTupleBuilder(fieldCount + 1) : new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
// Perform bulk load.
- IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples, false);
+ IIndexBulkLoader bulkLoader =
+ ctx.getIndex().createBulkLoader(0.7f, false, numTuples, false, NoOpPageWriteCallback.INSTANCE);
int c = 1;
for (CheckTuple checkTuple : checkTuples) {
if (LOGGER.isInfoEnabled()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 4eaa045..48ae036 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -53,6 +53,7 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.Test;
@@ -807,7 +808,8 @@
LOGGER.info("Bulk loading " + numInserts + " tuples");
}
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts, true);
+ IIndexBulkLoader bulkLoader =
+ treeIndex.createBulkLoader(0.7f, false, numInserts, true, NoOpPageWriteCallback.INSTANCE);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
index 16ce33c..c969526 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.storage.am.bloomfilter.util.AbstractBloomFilterTest;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -71,7 +72,7 @@
bf.create();
bf.activate();
IIndexBulkLoader builder = bf.createBuilder(numElements, bloomFilterSpec.getNumHashes(),
- bloomFilterSpec.getNumBucketsPerElements());
+ bloomFilterSpec.getNumBucketsPerElements(), NoOpPageWriteCallback.INSTANCE);
int fieldCount = 2;
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
@@ -130,7 +131,7 @@
bf.create();
bf.activate();
IIndexBulkLoader builder = bf.createBuilder(numElements, bloomFilterSpec.getNumHashes(),
- bloomFilterSpec.getNumBucketsPerElements());
+ bloomFilterSpec.getNumBucketsPerElements(), NoOpPageWriteCallback.INSTANCE);
int fieldCount = 5;
ISerializerDeserializer[] fieldSerdes = { new UTF8StringSerializerDeserializer(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
index 75437c8..15774c9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
@@ -45,6 +45,7 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.junit.Assert;
import org.junit.FixMethodOrder;
import org.junit.Test;
@@ -174,7 +175,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(FIELD_COUNT);
ArrayTupleReference tuple = new ArrayTupleReference();
- IIndexBulkLoader bulkloader = btree.createBulkLoader(1, true, 0, true);
+ IIndexBulkLoader bulkloader = btree.createBulkLoader(1, true, 0, true, NoOpPageWriteCallback.INSTANCE);
// insert keys into btree
for (int i = 0; i < keys.size(); i++) {
TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index 3043669..28b0139 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -57,7 +57,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
index 251f57a..b4e4d84 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -81,7 +81,8 @@
return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- scheduler, ioCallbackFactory, harness.getMetadataPageManagerFactory(), false, true, false);
+ scheduler, ioCallbackFactory, harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory(), false, true, false);
}
private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys)
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index 2994cbc..857a20c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -57,7 +57,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index 83d9bf1..deee002 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -61,9 +61,9 @@
return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), true,
- filterTypeTraits, filterCmpFactories, btreeFields, filterFields, true,
- harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+ harness.getPageWriteCallbackFactory(), true, filterTypeTraits, filterCmpFactories, btreeFields,
+ filterFields, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
index 03b221e..a6ba178 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
@@ -56,7 +56,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, 1,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
ctx.getIndex().create();
ctx.getIndex().activate();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
index e348405..7778308 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
@@ -56,7 +56,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), filtered, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index 7077631..69760bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -57,7 +57,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index 07b6ed3..431abb8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -61,7 +61,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, fieldSerdes.length,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
index = testCtx.getIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index a8639c3..f3ae8a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -104,8 +104,8 @@
return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- scheduler, harness.getIOOperationCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered,
- true, false);
+ scheduler, harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory(), filtered, true, false);
}
private class TestIoScheduler implements ILSMIOOperationScheduler {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index f5cf15c..c9a2a99 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -56,7 +56,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), filtered, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index d83f475..0d11607 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -52,8 +52,8 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
- harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
+ null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index 391711c..07cd08f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -58,7 +58,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java
new file mode 100644
index 0000000..c0dc6b0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.btree;
+
+import java.util.Random;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestDriver;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexPageWriteCallback;
+import org.apache.hyracks.storage.common.IResource;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreePageWriteCallbackTest extends OrderedIndexTestDriver {
+
+ private final OrderedIndexTestUtils orderedIndexTestUtils;
+
+ private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+ private final int PAGES_PER_FORCE = 16;
+
+ private LSMIndexPageWriteCallback lastCallback = null;
+
+ private final ILSMPageWriteCallbackFactory pageWriteCallbackFactory = new ILSMPageWriteCallbackFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) {
+ // no op
+ }
+
+ @Override
+ public IPageWriteCallback createPageWriteCallback() throws HyracksDataException {
+ lastCallback = new LSMIndexPageWriteCallback(PAGES_PER_FORCE);
+ return lastCallback;
+ }
+ };
+
+ public LSMBTreePageWriteCallbackTest() {
+ super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST);
+ this.orderedIndexTestUtils = new OrderedIndexTestUtils();
+
+ }
+
+ @Before
+ public void setUp() throws HyracksDataException {
+ harness.setUp();
+ }
+
+ @After
+ public void tearDown() throws HyracksDataException {
+ harness.tearDown();
+ }
+
+ @Override
+ protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+ BTreeLeafFrameType leafType, boolean filtered) throws Exception {
+ return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
+ harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), pageWriteCallbackFactory,
+ harness.getMetadataPageManagerFactory(), false, true, false);
+ }
+
+ @Override
+ protected Random getRandom() {
+ return harness.getRandom();
+ }
+
+ @Override
+ protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
+ ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey,
+ ITupleReference prefixHighKey) throws Exception {
+ OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType, false);
+ ctx.getIndex().create();
+ ctx.getIndex().activate();
+ // Start off with one tree bulk loaded.
+ // We assume all fieldSerdes are of the same type. Check the first one
+ // to determine which field types to generate.
+ if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+ orderedIndexTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+ } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
+ orderedIndexTestUtils.bulkLoadStringTuples(ctx, numTuplesToInsert, getRandom());
+ }
+
+ int maxTreesToMerge = AccessMethodTestsConfig.LSM_BTREE_MAX_TREES_TO_MERGE;
+ for (int i = 0; i < maxTreesToMerge; i++) {
+ for (int j = 0; j < i; j++) {
+ if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+ orderedIndexTestUtils.insertIntTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
+ } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
+ orderedIndexTestUtils.insertStringTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
+ }
+ }
+ ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
+ ILSMIOOperation mergeOp = accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
+ mergeOp.addCompleteListener(op -> {
+ if (op.getIOOpertionType() == LSMIOOperationType.MERGE) {
+ long numPages = op.getNewComponent().getComponentSize()
+ / harness.getDiskBufferCache().getPageSizeWithHeader() - 1;
+ // we skipped the metadata page for simplicity
+ Assert.assertEquals(numPages / PAGES_PER_FORCE, lastCallback.getTotalForces());
+ }
+ });
+
+ }
+ ctx.getIndex().deactivate();
+ ctx.getIndex().destroy();
+ }
+
+ @Override
+ protected String getTestOpName() {
+ return "Write Page Callback";
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index bdfd458..b47f5dd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -80,7 +80,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 59c9ebb..56bd14c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -61,8 +61,8 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
- harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
+ null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
}
@@ -292,7 +292,8 @@
throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
}
- IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin, true);
+ IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin, true,
+ harness.getPageWriteCallbackFactory().createPageWriteCallback());
for (int i = begin; i <= end; i++) {
TupleUtils.createIntegerTuple(builder, tuple, i);
bulkloader.add(tuple);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index afcb3c1..bbe0966 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -123,7 +123,8 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, !hasOnlyKeys, hasOnlyKeys);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, !hasOnlyKeys,
+ hasOnlyKeys);
}
interface IndexModification {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index c14474b..a695798 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -74,8 +74,8 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), true, null, null, null, null, true,
- harness.getMetadataPageManagerFactory(), true, ITracer.NONE,
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
+ null, true, harness.getMetadataPageManagerFactory(), true, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index 17d9bcf..ceb8c11 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -57,7 +57,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), false, true, false);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
index 519f895..2d0f079 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
@@ -42,6 +42,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
@@ -86,15 +87,15 @@
ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOperationCallbackFactory, boolean needKeyDupCheck, boolean hasBloomFilter,
+ ILSMIOOperationCallbackFactory ioOperationCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean needKeyDupCheck, boolean hasBloomFilter,
int[] btreeFields, int[] filterFields, boolean durable, boolean updateAware, ITracer tracer)
throws HyracksDataException {
super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
diskBufferCache, fileManager, componentFactory, bulkLoadComponentFactory, filterHelper,
filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy,
- opTracker, ioScheduler, ioOperationCallbackFactory, needKeyDupCheck, hasBloomFilter, btreeFields,
- filterFields, durable, updateAware, tracer);
-
+ opTracker, ioScheduler, ioOperationCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck,
+ hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer);
addModifyCallback(AllowTestOpCallback.INSTANCE);
addSearchCallback(AllowTestOpCallback.INSTANCE);
addFlushCallback(AllowTestOpCallback.INSTANCE);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index baa95e9..816bd0e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -53,12 +54,14 @@
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
- btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, durable, NoOpCompressorDecompressorFactory.INSTANCE);
+ btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
+ metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
@@ -80,12 +83,13 @@
}
}
ioOpCallbackFactory.initialize(serviceCtx, this);
+ pageWriteCallbackFactory.initialize(serviceCtx, this);
return TestLsmBtreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx),
typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
- ioOpCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
- durable, metadataPageManagerFactory, false, serviceCtx.getTracer());
+ ioOpCallbackFactory, pageWriteCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories,
+ btreeFields, filterFields, durable, metadataPageManagerFactory, false, serviceCtx.getTracer());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 3d7c520..a6c1a23 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -29,6 +29,7 @@
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -41,14 +42,16 @@
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
- opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE);
+ opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
+ bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
@Override
@@ -56,7 +59,7 @@
return new TestLsmBtreeLocalResource(typeTraits, cmpFactories, bloomFilterKeyFields,
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
- opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
- durable);
+ opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+ vbcProvider, ioSchedulerProvider, durable);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
index 82f7959..eebfb2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
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.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.frames.LSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.ComponentFilterHelper;
@@ -62,10 +63,10 @@
FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits,
- IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields, boolean durable,
- IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer)
- throws HyracksDataException {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
+ boolean updateAware, ITracer tracer) throws HyracksDataException {
LSMBTreeTupleWriterFactory insertTupleWriterFactory =
new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
@@ -115,7 +116,7 @@
return new TestLsmBtree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
- cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, needKeyDupCheck, hasBloomFilter,
- btreeFields, filterFields, durable, updateAware, tracer);
+ cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
+ needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index 2e7a9f0..63a51b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -28,7 +28,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
import org.apache.hyracks.storage.common.file.IFileMapManager;
public class TestVirtualBufferCache implements IVirtualBufferCache {
@@ -155,13 +157,8 @@
}
@Override
- public IFIFOPageQueue createFIFOQueue() {
- return vbc.createFIFOQueue();
- }
-
- @Override
- public void finishQueue() throws HyracksDataException {
- vbc.finishQueue();
+ public IFIFOPageWriter createFIFOWriter(IPageWriteCallback callback, IPageWriteFailureCallback failureCallback) {
+ return vbc.createFIFOWriter(callback, failureCallback);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index 85038c7..26707d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -57,8 +57,9 @@
return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), true,
- null, null, null, null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+ harness.getPageWriteCallbackFactory(), true, null, null, null, null, true,
+ harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
index 3638a4e..4f1237e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.storage.am.common.datagen.DataGenThread;
import org.apache.hyracks.storage.am.common.datagen.TupleBatch;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
public class BTreeBulkLoadRunner extends BTreeRunner {
@@ -40,7 +41,7 @@
public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
btree.create();
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L, true);
+ IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L, true, NoOpPageWriteCallback.INSTANCE);
for (int i = 0; i < numBatches; i++) {
TupleBatch batch = dataGen.tupleBatchQueue.take();
for (int j = 0; j < batch.size(); j++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 3bb1fa4..75a385e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -124,9 +125,10 @@
lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
- new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallbackFactory.INSTANCE, true, null, null,
- null, null, true, TestStorageManagerComponentHolder.getMetadataPageManagerFactory(), false,
- ITracer.NONE, NoOpCompressorDecompressorFactory.INSTANCE);
+ new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallbackFactory.INSTANCE,
+ NoOpPageWriteCallbackFactory.INSTANCE, true, null, null, null, null, true,
+ TestStorageManagerComponentHolder.getMetadataPageManagerFactory(), false, ITracer.NONE,
+ NoOpCompressorDecompressorFactory.INSTANCE);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index e38514d..396bf33 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -39,6 +39,7 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
@@ -79,6 +80,7 @@
FileReference file, IBufferCache diskBufferCache, ISerializerDeserializer[] fieldSerdes, int numKeyFields,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, boolean filtered, boolean needKeyDupCheck,
boolean updateAware) throws HyracksDataException {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
@@ -99,14 +101,14 @@
IBinaryComparatorFactory[] filterCmp = { cmpFactories[0] };
lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, needKeyDupCheck, filterTypeTraits, filterCmp, btreefields,
- filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE,
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, filterTypeTraits,
+ filterCmp, btreefields, filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE,
NoOpCompressorDecompressorFactory.INSTANCE);
} else {
lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, needKeyDupCheck, null, null, null, null, true,
- metadataPageManagerFactory,
+ ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, null, null, null, null,
+ true, metadataPageManagerFactory,
updateAware, new Tracer(LSMBTreeTestContext.class.getSimpleName(),
ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
NoOpCompressorDecompressorFactory.INSTANCE);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 707628b..d64ea30 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -41,8 +41,10 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
@@ -79,6 +81,7 @@
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
protected ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ protected ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
protected IMetadataPageManagerFactory metadataPageManagerFactory;
protected final Random rnd = new Random();
@@ -101,6 +104,7 @@
this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
+ this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
}
public void setUp() throws HyracksDataException {
@@ -213,6 +217,10 @@
return ioOpCallbackFactory;
}
+ public ILSMPageWriteCallbackFactory getPageWriteCallbackFactory() {
+ return pageWriteCallbackFactory;
+ }
+
public IMetadataPageManagerFactory getMetadataPageManagerFactory() {
return metadataPageManagerFactory;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index c1e1b4c..c220697 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -39,10 +39,12 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
@@ -73,6 +75,7 @@
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
protected ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ protected ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
protected final Random rnd = new Random();
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -96,6 +99,7 @@
this.mergePolicy = new NoMergePolicy();
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+ this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
this.numMutableComponents = AccessMethodTestsConfig.LSM_INVINDEX_NUM_MUTABLE_COMPONENTS;
}
@@ -217,6 +221,10 @@
return ioOpCallbackFactory;
}
+ public ILSMPageWriteCallbackFactory getPageWriteCallbackFactory() {
+ return pageWriteCallbackFactory;
+ }
+
public IMetadataPageManagerFactory getMetadataPageManagerFactory() {
return metadataPageManagerFactory;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 164efc2..524904f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -185,9 +185,10 @@
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
- filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+ harness.getMetadataPageManagerFactory(),
new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
break;
@@ -198,9 +199,10 @@
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
- filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+ harness.getMetadataPageManagerFactory(),
new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
break;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index b67d909..18ee1f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -106,6 +106,7 @@
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.test.support.TestUtils;
import org.apache.hyracks.util.IThreadStats;
import org.apache.hyracks.util.IThreadStatsCollector;
@@ -243,7 +244,8 @@
ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
// Use the expected index to bulk-load the actual index.
- IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs, true);
+ IIndexBulkLoader bulkLoader =
+ testCtx.getIndex().createBulkLoader(1.0f, false, numDocs, true, NoOpPageWriteCallback.INSTANCE);
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(testCtx.getFieldSerdes().length);
ArrayTupleReference tuple = new ArrayTupleReference();
Iterator<CheckTuple> checkTupleIter = tmpMemIndex.iterator();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index 7c088d9..21b6a70 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -60,7 +60,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index e59ca1d..0662540 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -60,7 +60,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index d64ff4c..3ca25ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -47,7 +47,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields, btreeFields,
filterTypeTraits, filterCmpFactories, filterFields, true, false,
harness.getMetadataPageManagerFactory());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 9c607a8..a48bfaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -60,7 +60,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index f12b35a..aa2bb63 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -70,7 +70,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
index = testCtx.getIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 512b050..864d555 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -59,7 +59,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
index 0df7caf..dcdc8b5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
@@ -60,7 +60,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory(), true);
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), true);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index d9328d5..1ac7d0b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -59,7 +59,8 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getMetadataPageManagerFactory());
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 1fa9d95..2d74823 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -60,7 +60,8 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getMetadataPageManagerFactory());
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index f10055e..75f656d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -47,6 +47,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+ harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields,
filterTypeTraits, filterCmpFactories, filterFields, true, false,
harness.getMetadataPageManagerFactory());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index c318623..4872e68 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -59,7 +59,8 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getMetadataPageManagerFactory());
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index 77cceeb..ccce785 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -70,7 +70,7 @@
harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
- harness.getMetadataPageManagerFactory());
+ harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
index = testCtx.getIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index 1149099..df52549 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -58,7 +58,8 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getMetadataPageManagerFactory());
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ harness.getMetadataPageManagerFactory());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 720480d..bbf6263 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -64,7 +64,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
null, null, true, false, harness.getMetadataPageManagerFactory());
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index c273d8d..7d6e2e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -65,6 +65,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+ harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
false, harness.getMetadataPageManagerFactory());
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index acdb268..2140558 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -36,6 +36,7 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
@@ -78,10 +79,11 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory) throws Exception {
return create(ioManager, virtualBufferCaches, file, diskBufferCache, fieldSerdes, valueProviderFactories,
numKeyFields, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackFactory, metadataPageManagerFactory, false);
+ ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, false);
}
public static LSMRTreeTestContext create(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
@@ -89,6 +91,7 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, boolean isPointMBR) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories =
@@ -105,8 +108,8 @@
LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache,
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
- null, null, true, isPointMBR, metadataPageManagerFactory);
+ pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory);
LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 598cf18..e189898 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -39,9 +39,11 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
@@ -70,6 +72,7 @@
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ protected ILSMPageWriteCallbackFactory pageWriteCallbackFactory;
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
protected IMetadataPageManagerFactory metadataPageManagerFactory =
@@ -93,6 +96,7 @@
this.mergePolicy = new NoMergePolicy();
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+ this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
this.numMutableComponents = AccessMethodTestsConfig.LSM_RTREE_NUM_MUTABLE_COMPONENTS;
}
@@ -209,6 +213,10 @@
return ioOpCallbackFactory;
}
+ public ILSMPageWriteCallbackFactory getPageWriteCallbackFactory() {
+ return pageWriteCallbackFactory;
+ }
+
public IMetadataPageManagerFactory getMetadataPageManagerFactory() {
return metadataPageManagerFactory;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index fcb00ed..f9d9046 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -36,6 +36,7 @@
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.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
@@ -78,8 +79,8 @@
ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory, IMetadataPageManagerFactory metadataPageManagerFactory)
- throws Exception {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ IMetadataPageManagerFactory metadataPageManagerFactory) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories =
SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -88,8 +89,8 @@
LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager,
virtualBufferCaches, file, diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
- false, metadataPageManagerFactory);
+ pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ null, null, null, null, true, false, metadataPageManagerFactory);
LSMRTreeWithAntiMatterTuplesTestContext testCtx =
new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes, lsmTree);
return testCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
index d0d02a9..593b00d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.test.support.TestUtils;
@@ -96,9 +97,8 @@
long dpid = BufferedFileHandle.getDiskPageId(fileId, i);
ICachedPage page = bufferCache.confiscatePage(dpid);
page.getBuffer().putInt(0, i);
- bufferCache.createFIFOQueue().put(page, HaltOnFailureCallback.INSTANCE);
+ bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, HaltOnFailureCallback.INSTANCE).write(page);
}
- bufferCache.finishQueue();
bufferCache.closeFile(fileId);
ExecutorService executor = Executors.newFixedThreadPool(bufferCacheNumPages);
MutableObject<Thread>[] readers = new MutableObject[bufferCacheNumPages];
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
index a978bc5..963b1d8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheWithCompressionTest.java
@@ -44,7 +44,8 @@
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.NoOpPageWriteCallback;
import org.apache.hyracks.storage.common.compression.SnappyCompressorDecompressorFactory;
import org.apache.hyracks.storage.common.compression.file.CompressedFileReference;
import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
@@ -99,15 +100,15 @@
final int numPages = 16;
bufferCache.openFile(fileId);
final ICompressedPageWriter writer = bufferCache.getCompressedPageWriter(fileId);
- final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
+ final IFIFOPageWriter pageWriter =
+ bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, HaltOnFailureCallback.INSTANCE);
for (int i = 0; i < numPages; i++) {
long dpid = BufferedFileHandle.getDiskPageId(fileId, i);
ICachedPage page = bufferCache.confiscatePage(dpid);
writer.prepareWrite(page);
page.getBuffer().putInt(0, i);
- queue.put(page, HaltOnFailureCallback.INSTANCE);
+ pageWriter.write(page);
}
- bufferCache.finishQueue();
writer.endWriting();
bufferCache.closeFile(fileId);
ExecutorService executor = Executors.newFixedThreadPool(bufferCacheNumPages);
@@ -205,9 +206,9 @@
for (int i = 0; i < num; i++) {
page.getBuffer().putInt(i * 4, i);
}
- final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
- queue.put(page, HaltOnFailureCallback.INSTANCE);
- bufferCache.finishQueue();
+ final IFIFOPageWriter pageWriter =
+ bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, HaltOnFailureCallback.INSTANCE);
+ pageWriter.write(page);
writer.endWriting();
bufferCache.closeFile(fileId);
@@ -259,9 +260,9 @@
values.add(x);
}
pageContents.put(fileId, values);
- final IFIFOPageQueue queue = bufferCache.createFIFOQueue();
- queue.put(page, HaltOnFailureCallback.INSTANCE);
- bufferCache.finishQueue();
+ final IFIFOPageWriter pageWriter =
+ bufferCache.createFIFOWriter(NoOpPageWriteCallback.INSTANCE, HaltOnFailureCallback.INSTANCE);
+ pageWriter.write(page);
writer.endWriting();
}