Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 7992A200CBB for ; Mon, 19 Jun 2017 21:16:12 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 78A3F160BCD; Mon, 19 Jun 2017 19:16:12 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 58EA3160BE4 for ; Mon, 19 Jun 2017 21:16:10 +0200 (CEST) Received: (qmail 23811 invoked by uid 500); 19 Jun 2017 19:16:09 -0000 Mailing-List: contact commits-help@asterixdb.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@asterixdb.apache.org Delivered-To: mailing list commits@asterixdb.apache.org Received: (qmail 23791 invoked by uid 99); 19 Jun 2017 19:16:09 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 19 Jun 2017 19:16:09 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CF04EE10BF; Mon, 19 Jun 2017 19:16:06 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: amoudi@apache.org To: commits@asterixdb.apache.org Date: Mon, 19 Jun 2017 19:16:12 -0000 Message-Id: In-Reply-To: <224e3c58704b46c297db68e48b3c2f6d@git.apache.org> References: <224e3c58704b46c297db68e48b3c2f6d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [7/7] asterixdb git commit: ASTERIXDB-1945 [STO] Cleanup Buffer Cache API archived-at: Mon, 19 Jun 2017 19:16:12 -0000 ASTERIXDB-1945 [STO] Cleanup Buffer Cache API Fix the buffer cache to follow the API such that: 1. createFile creates the file. 2. deleteFile deletes the file. 3. openFile opens the file. 4. closeFile closes the file. 5. creates existing file is not allowed. 6. deletes deleted file is not allowed. 7. open non existing file is not allowed. In addition, we hide the file map from all other components. Change-Id: I15565b07afdc94ac74c608bfe4480fa09dcf8f1c Reviewed-on: https://asterix-gerrit.ics.uci.edu/1840 Tested-by: Jenkins BAD: Jenkins Integration-Tests: Jenkins Reviewed-by: Murtadha Hubail Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/ae3daf6e Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/ae3daf6e Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/ae3daf6e Branch: refs/heads/master Commit: ae3daf6ef3397e583637360dc460c6391e03dc29 Parents: 8ba5944 Author: Abdullah Alamoudi Authored: Mon Jun 19 11:01:40 2017 -0700 Committer: abdullah alamoudi Committed: Mon Jun 19 12:15:42 2017 -0700 ---------------------------------------------------------------------- .../AppRuntimeContextProviderForRecovery.java | 10 +- .../asterix/app/nc/NCAppRuntimeContext.java | 18 +- .../hyracks/bootstrap/NCApplication.java | 10 +- .../common/api/INcApplicationContext.java | 3 - .../asterix/common/context/FileMapManager.java | 85 --------- .../LSMBTreeIOOperationCallback.java | 2 +- .../LSMBTreeWithBuddyIOOperationCallback.java | 2 +- .../LSMRTreeIOOperationCallback.java | 2 +- .../IAppRuntimeContextProvider.java | 27 ++- .../storage/LSMComponentProperties.java | 3 +- .../utils/RuntimeComponentsProvider.java | 6 - .../locking/TestRuntimeContextProvider.java | 10 +- hyracks-fullstack/hyracks/hyracks-api/pom.xml | 4 + .../hyracks/api/exceptions/ErrorCode.java | 18 +- .../apache/hyracks/api/io/FileReference.java | 4 +- .../org/apache/hyracks/api/io/IFileHandle.java | 7 +- .../org/apache/hyracks/api/io/IIOManager.java | 4 +- .../org/apache/hyracks/api/util/IoUtil.java | 92 ++++++++++ .../src/main/resources/errormsg/en.properties | 18 +- .../hyracks/control/nc/io/FileHandle.java | 15 +- .../apache/hyracks/control/nc/io/IOManager.java | 13 +- .../btree/helper/BTreeHelperStorageManager.java | 6 - .../examples/btree/helper/RuntimeContext.java | 4 +- .../am/bloomfilter/impls/BloomFilter.java | 81 ++++----- .../bloomfilter/impls/BloomFilterFactory.java | 7 +- .../am/btree/dataflow/BTreeResource.java | 5 +- .../hyracks/storage/am/btree/impls/BTree.java | 47 +++-- .../storage/am/btree/util/BTreeUtils.java | 23 ++- .../storage/am/common/build/IndexBuilder.java | 18 ++ .../TreeIndexStatsOperatorNodePushable.java | 6 +- .../am/common/impls/AbstractTreeIndex.java | 90 ++-------- .../common/util/TreeIndexBufferCacheWarmup.java | 16 +- .../am/common/util/TreeIndexStatsGatherer.java | 19 +- .../dataflow/ExternalBTreeLocalResource.java | 6 +- .../ExternalBTreeWithBuddyLocalResource.java | 2 +- .../btree/dataflow/LSMBTreeLocalResource.java | 6 +- .../am/lsm/btree/impls/ExternalBTree.java | 40 ++--- .../lsm/btree/impls/ExternalBTreeWithBuddy.java | 28 ++- .../storage/am/lsm/btree/impls/LSMBTree.java | 39 ++-- .../am/lsm/btree/impls/LSMBTreeFileManager.java | 40 ++--- .../impls/LSMBTreeWithBuddyFileManager.java | 51 +++--- .../am/lsm/btree/utils/LSMBTreeUtil.java | 99 +++++----- .../hyracks-storage-am-lsm-common/pom.xml | 5 +- .../am/lsm/common/api/ILSMIndexFileManager.java | 22 +-- .../impls/AbortRecoverLSMIndexFileManager.java | 2 +- .../am/lsm/common/impls/AbstractLSMIndex.java | 31 +--- .../impls/AbstractLSMIndexFileManager.java | 125 ++++++------- .../am/lsm/common/impls/BTreeFactory.java | 15 +- .../am/lsm/common/impls/IndexFactory.java | 6 +- .../impls/MultitenantVirtualBufferCache.java | 38 +++- .../am/lsm/common/impls/TreeIndexFactory.java | 9 +- .../am/lsm/common/impls/VirtualBufferCache.java | 51 ++++-- .../pom.xml | 3 +- .../lsm/invertedindex/api/IInvertedIndex.java | 9 + .../dataflow/LSMInvertedIndexLocalResource.java | 19 +- .../invertedindex/impls/LSMInvertedIndex.java | 40 ++--- .../impls/LSMInvertedIndexFileManager.java | 32 ++-- .../impls/PartitionedLSMInvertedIndex.java | 19 +- .../inmemory/InMemoryInvertedIndex.java | 11 +- .../ondisk/OnDiskInvertedIndex.java | 105 +++-------- .../ondisk/OnDiskInvertedIndexFactory.java | 10 +- .../ondisk/PartitionedOnDiskInvertedIndex.java | 14 +- .../PartitionedOnDiskInvertedIndexFactory.java | 18 +- .../invertedindex/util/InvertedIndexUtils.java | 113 ++++++------ .../dataflow/ExternalRTreeLocalResource.java | 5 +- .../rtree/dataflow/LSMRTreeLocalResource.java | 6 +- .../LSMRTreeWithAntiMatterLocalResource.java | 4 +- .../am/lsm/rtree/impls/AbstractLSMRTree.java | 56 +++--- .../am/lsm/rtree/impls/ExternalRTree.java | 53 ++---- .../storage/am/lsm/rtree/impls/LSMRTree.java | 42 ++--- .../am/lsm/rtree/impls/LSMRTreeFileManager.java | 51 +++--- .../impls/LSMRTreeWithAntiMatterTuples.java | 25 ++- ...LSMRTreeWithAntiMatterTuplesFileManager.java | 6 +- .../am/lsm/rtree/impls/RTreeFactory.java | 11 +- .../am/lsm/rtree/utils/LSMRTreeUtils.java | 90 +++++----- .../am/rtree/dataflow/RTreeResource.java | 5 +- .../hyracks/storage/am/rtree/impls/RTree.java | 45 +++-- .../storage/am/rtree/util/RTreeUtils.java | 19 +- .../hyracks/storage/common/IStorageManager.java | 8 - .../storage/common/buffercache/BufferCache.java | 179 +++++++++++-------- .../common/buffercache/DebugBufferCache.java | 21 ++- .../common/buffercache/IBufferCache.java | 62 ++++++- .../storage/common/file/FileMapManager.java | 84 +++++++++ .../storage/common/file/IFileMapManager.java | 6 +- .../common/file/TransientFileMapManager.java | 75 -------- .../am/common/AbstractIndexLifecycleTest.java | 60 +++++-- .../test/support/TestStorageManager.java | 6 - .../TestStorageManagerComponentHolder.java | 6 +- .../storage/am/bloomfilter/BloomFilterTest.java | 14 +- .../util/BloomFilterTestHarness.java | 7 - .../storage/am/btree/BTreeBulkLoadTest.java | 7 +- .../storage/am/btree/BTreeDeleteTest.java | 7 +- .../storage/am/btree/BTreeExamplesTest.java | 2 +- .../storage/am/btree/BTreeInsertTest.java | 15 +- .../storage/am/btree/BTreeLifecycleTest.java | 15 +- .../BTreeModificationOperationCallbackTest.java | 6 +- .../storage/am/btree/BTreeSearchCursorTest.java | 32 ++-- .../btree/BTreeSearchOperationCallbackTest.java | 7 +- .../storage/am/btree/BTreeSortedInsertTest.java | 7 +- .../storage/am/btree/BTreeStatsTest.java | 11 +- .../storage/am/btree/BTreeUpdateSearchTest.java | 4 +- .../storage/am/btree/BTreeUpdateTest.java | 7 +- .../storage/am/btree/BTreeUpsertTest.java | 15 +- .../storage/am/btree/FieldPrefixNSMTest.java | 5 +- .../storage/am/btree/StorageFileAccessTest.java | 8 +- .../btree/multithread/BTreeMultiThreadTest.java | 7 +- .../storage/am/btree/util/BTreeTestContext.java | 11 +- .../storage/am/btree/util/BTreeTestHarness.java | 7 - .../am/lsm/btree/LSMBTreeBulkLoadTest.java | 10 +- .../am/lsm/btree/LSMBTreeDeleteTest.java | 10 +- .../am/lsm/btree/LSMBTreeExamplesTest.java | 4 +- .../am/lsm/btree/LSMBTreeFileManagerTest.java | 15 +- .../am/lsm/btree/LSMBTreeFilterMergeTest.java | 10 +- .../am/lsm/btree/LSMBTreeInsertTest.java | 12 +- .../am/lsm/btree/LSMBTreeLifecycleTest.java | 17 +- .../storage/am/lsm/btree/LSMBTreeMergeTest.java | 10 +- ...MBTreeModificationOperationCallbackTest.java | 3 +- .../am/lsm/btree/LSMBTreeMultiBulkLoadTest.java | 10 +- .../btree/LSMBTreeScanDiskComponentsTest.java | 10 +- .../LSMBTreeSearchOperationCallbackTest.java | 10 +- .../am/lsm/btree/LSMBTreeUpdateTest.java | 12 +- .../multithread/LSMBTreeMultiThreadTest.java | 4 +- .../storage/am/lsm/btree/perf/BTreeRunner.java | 4 +- .../am/lsm/btree/perf/InMemoryBTreeRunner.java | 5 +- .../am/lsm/btree/perf/LSMTreeRunner.java | 4 +- .../am/lsm/btree/util/LSMBTreeTestContext.java | 36 ++-- .../am/lsm/btree/util/LSMBTreeTestHarness.java | 7 - .../component/TestLsmIndexFileManager.java | 5 +- .../common/test/LSMIndexFileManagerTest.java | 12 +- .../lsm/common/test/VirtualBufferCacheTest.java | 2 +- .../common/LSMInvertedIndexTestHarness.java | 7 - .../OnDiskInvertedIndexLifecycleTest.java | 25 +-- .../util/LSMInvertedIndexTestContext.java | 32 ++-- .../am/lsm/rtree/LSMRTreeBulkLoadTest.java | 7 +- .../am/lsm/rtree/LSMRTreeDeleteTest.java | 11 +- .../am/lsm/rtree/LSMRTreeExamplesTest.java | 8 +- .../am/lsm/rtree/LSMRTreeInsertTest.java | 11 +- .../am/lsm/rtree/LSMRTreeLifecycleTest.java | 21 +-- .../storage/am/lsm/rtree/LSMRTreeMergeTest.java | 10 +- ...SMRTreeWithAntiMatterTuplesBulkLoadTest.java | 7 +- .../LSMRTreeWithAntiMatterTuplesDeleteTest.java | 7 +- ...SMRTreeWithAntiMatterTuplesExamplesTest.java | 4 +- .../LSMRTreeWithAntiMatterTuplesInsertTest.java | 7 +- ...MRTreeWithAntiMatterTuplesLifecycleTest.java | 20 +-- .../LSMRTreeWithAntiMatterTuplesMergeTest.java | 7 +- .../multithread/LSMRTreeMultiThreadTest.java | 8 +- ...TreeWithAntiMatterTuplesMultiThreadTest.java | 4 +- .../am/lsm/rtree/util/LSMRTreeTestContext.java | 15 +- .../am/lsm/rtree/util/LSMRTreeTestHarness.java | 7 - ...LSMRTreeWithAntiMatterTuplesTestContext.java | 22 +-- .../storage/am/rtree/RTreeBulkLoadTest.java | 5 +- .../storage/am/rtree/RTreeDeleteTest.java | 5 +- .../storage/am/rtree/RTreeExamplesTest.java | 5 +- .../storage/am/rtree/RTreeInsertTest.java | 5 +- .../storage/am/rtree/RTreeLifecycleTest.java | 15 +- .../storage/am/rtree/RTreeSearchCursorTest.java | 4 +- .../rtree/multithread/RTreeMultiThreadTest.java | 5 +- .../am/rtree/utils/RTreeTestContext.java | 12 +- .../am/rtree/utils/RTreeTestHarness.java | 7 - .../common/BufferCacheRegressionTest.java | 21 ++- .../hyracks/storage/common/BufferCacheTest.java | 29 +-- .../java/org/apache/hyracks/util/JSONUtil.java | 43 +++++ 162 files changed, 1622 insertions(+), 1788 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java index 397b459..1fea840 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java @@ -19,9 +19,9 @@ package org.apache.asterix.api.common; import org.apache.asterix.app.nc.NCAppRuntimeContext; -import org.apache.asterix.common.api.ThreadExecutor; -import org.apache.asterix.common.api.INcApplicationContext; import org.apache.asterix.common.api.IDatasetLifecycleManager; +import org.apache.asterix.common.api.INcApplicationContext; +import org.apache.asterix.common.api.ThreadExecutor; import org.apache.asterix.common.transactions.IAppRuntimeContextProvider; import org.apache.asterix.common.transactions.ITransactionSubsystem; import org.apache.hyracks.api.io.IIOManager; @@ -29,7 +29,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker; import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public class AppRuntimeContextProviderForRecovery implements IAppRuntimeContextProvider { @@ -45,11 +44,6 @@ public class AppRuntimeContextProviderForRecovery implements IAppRuntimeContextP } @Override - public IFileMapProvider getFileMapManager() { - return asterixAppRuntimeContext.getFileMapManager(); - } - - @Override public ITransactionSubsystem getTransactionSubsystem() { return asterixAppRuntimeContext.getTransactionSubsystem(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java index c23da6c..55b9adc 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java @@ -47,7 +47,6 @@ import org.apache.asterix.common.config.ReplicationProperties; import org.apache.asterix.common.config.StorageProperties; import org.apache.asterix.common.config.TransactionProperties; import org.apache.asterix.common.context.DatasetLifecycleManager; -import org.apache.asterix.common.context.FileMapManager; import org.apache.asterix.common.context.IStorageComponentProvider; import org.apache.asterix.common.exceptions.ACIDException; import org.apache.asterix.common.exceptions.AsterixException; @@ -94,8 +93,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache; import org.apache.hyracks.storage.common.buffercache.ICacheMemoryAllocator; import org.apache.hyracks.storage.common.buffercache.IPageCleanerPolicy; import org.apache.hyracks.storage.common.buffercache.IPageReplacementStrategy; -import org.apache.hyracks.storage.common.file.IFileMapManager; -import org.apache.hyracks.storage.common.file.IFileMapProvider; +import org.apache.hyracks.storage.common.file.FileMapManager; import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory; import org.apache.hyracks.storage.common.file.IResourceIdFactory; @@ -117,7 +115,6 @@ public class NCAppRuntimeContext implements INcApplicationContext { private final NodeProperties nodeProperties; private ThreadExecutor threadExecutor; private IDatasetLifecycleManager datasetLifecycleManager; - private IFileMapManager fileMapManager; private IBufferCache bufferCache; private ITransactionSubsystem txnSubsystem; private IMetadataNode metadataNodeStub; @@ -168,7 +165,6 @@ public class NCAppRuntimeContext implements INcApplicationContext { public void initialize(boolean initialRun) throws IOException, ACIDException { ioManager = getServiceContext().getIoManager(); threadExecutor = new ThreadExecutor(getServiceContext().getThreadFactory()); - fileMapManager = new FileMapManager(ioManager); ICacheMemoryAllocator allocator = new HeapBufferAllocator(); IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000); IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, @@ -236,16 +232,15 @@ public class NCAppRuntimeContext implements INcApplicationContext { //initialize replication channel replicationChannel = new ReplicationChannel(nodeId, replicationProperties, txnSubsystem.getLogManager(), - replicaResourcesManager, replicationManager, getServiceContext(), - asterixAppRuntimeContextProvider); + replicaResourcesManager, replicationManager, getServiceContext(), asterixAppRuntimeContextProvider); remoteRecoveryManager = new RemoteRecoveryManager(replicationManager, this, replicationProperties); - bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager, + bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(), storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory(), replicationManager); } else { - bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager, + bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(), storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory()); } @@ -302,11 +297,6 @@ public class NCAppRuntimeContext implements INcApplicationContext { } @Override - public IFileMapProvider getFileMapManager() { - return fileMapManager; - } - - @Override public ITransactionSubsystem getTransactionSubsystem() { return txnSubsystem; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java index 7c8e153..0f6b396 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java @@ -46,16 +46,17 @@ import org.apache.asterix.event.schema.cluster.Node; import org.apache.asterix.messaging.MessagingChannelInterfaceFactory; import org.apache.asterix.messaging.NCMessageBroker; import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository; -import org.apache.commons.io.FileUtils; import org.apache.hyracks.api.application.INCServiceContext; import org.apache.hyracks.api.application.IServiceContext; import org.apache.hyracks.api.client.ClusterControllerInfo; import org.apache.hyracks.api.client.HyracksConnection; import org.apache.hyracks.api.client.IHyracksClientConnection; import org.apache.hyracks.api.config.IConfigManager; +import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.IFileDeviceResolver; import org.apache.hyracks.api.job.resource.NodeCapacity; import org.apache.hyracks.api.messages.IMessageBroker; +import org.apache.hyracks.api.util.IoUtil; import org.apache.hyracks.control.common.controllers.NCConfig; import org.apache.hyracks.control.nc.BaseNCApplication; import org.apache.hyracks.control.nc.NodeControllerService; @@ -214,7 +215,7 @@ public class NCApplication extends BaseNCApplication { return new NodeCapacity(memorySize, maximumCoresForComputation); } - private void performLocalCleanUp() { + private void performLocalCleanUp() throws HyracksDataException { //Delete working area files from failed jobs runtimeContext.getIoManager().deleteWorkspaceFiles(); @@ -225,7 +226,10 @@ public class NCApplication extends BaseNCApplication { for (String ioDevice : ioDevices) { String tempDatasetsDir = ioDevice + storageDirName + File.separator + StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER; - FileUtils.deleteQuietly(new File(tempDatasetsDir)); + File tmpDsDir = new File(tempDatasetsDir); + if (tmpDsDir.exists()) { + IoUtil.delete(tmpDsDir); + } } //TODO http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java index a4b994b..d4b9a92 100644 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java +++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java @@ -38,7 +38,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory; import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker; import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; import org.apache.hyracks.storage.common.file.IResourceIdFactory; public interface INcApplicationContext extends IApplicationContext { @@ -59,8 +58,6 @@ public interface INcApplicationContext extends IApplicationContext { IBufferCache getBufferCache(); - IFileMapProvider getFileMapManager(); - ILocalResourceRepository getLocalResourceRepository(); IDatasetLifecycleManager getDatasetLifecycleManager(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/FileMapManager.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/FileMapManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/FileMapManager.java deleted file mode 100644 index c4a294d..0000000 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/FileMapManager.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.asterix.common.context; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.hyracks.api.exceptions.HyracksDataException; -import org.apache.hyracks.api.io.FileReference; -import org.apache.hyracks.api.io.IIOManager; -import org.apache.hyracks.storage.common.file.IFileMapManager; - -public class FileMapManager implements IFileMapManager { - - private static final long serialVersionUID = 1L; - private final transient IIOManager ioManager; - private final Map id2nameMap = new HashMap<>(); - private final Map name2IdMap = new HashMap<>(); - private int idCounter = 0; - - public FileMapManager(IIOManager ioManager) { - this.ioManager = ioManager; - } - - @Override - public FileReference lookupFileName(int fileId) throws HyracksDataException { - String fName = id2nameMap.get(fileId); - if (fName == null) { - throw new HyracksDataException("No mapping found for id: " + fileId); - } - return ioManager.resolveAbsolutePath(fName); - } - - @Override - public int lookupFileId(FileReference fileRef) throws HyracksDataException { - String fileName = fileRef.getFile().getAbsolutePath(); - Integer fileId = name2IdMap.get(fileName); - if (fileId == null) { - throw new HyracksDataException("No mapping found for name: " + fileName); - } - return fileId; - } - - @Override - public boolean isMapped(FileReference fileRef) { - String fileName = fileRef.getFile().getAbsolutePath(); - return name2IdMap.containsKey(fileName); - } - - @Override - public boolean isMapped(int fileId) { - return id2nameMap.containsKey(fileId); - } - - @Override - public void unregisterFile(int fileId) throws HyracksDataException { - String fileName = id2nameMap.remove(fileId); - name2IdMap.remove(fileName); - } - - @Override - public void registerFile(FileReference fileRef) throws HyracksDataException { - Integer fileId = idCounter++; - String fileName = fileRef.getFile().getAbsolutePath(); - id2nameMap.put(fileId, fileName); - name2IdMap.put(fileName, fileId); - } - -} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java index b9442dc..8c91879 100644 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java +++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java @@ -34,7 +34,7 @@ public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback @Override public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath) throws HyracksDataException { - if (diskComponentFilePath.endsWith(LSMBTreeFileManager.BTREE_STRING)) { + if (diskComponentFilePath.endsWith(LSMBTreeFileManager.BTREE_SUFFIX)) { LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) diskComponent; IMetadataPageManager metadataPageManager = (IMetadataPageManager) btreeComponent.getBTree().getPageManager(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java index 0e80397..9f50f57 100644 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java +++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java @@ -29,7 +29,7 @@ public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperation @Override public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath) throws HyracksDataException { - if (diskComponentFilePath.endsWith(LSMBTreeWithBuddyFileManager.BTREE_STRING)) { + if (diskComponentFilePath.endsWith(LSMBTreeWithBuddyFileManager.BTREE_SUFFIX)) { LSMBTreeWithBuddyDiskComponent btreeComponent = (LSMBTreeWithBuddyDiskComponent) diskComponent; IMetadataPageManager metadataPageManager = (IMetadataPageManager) btreeComponent.getBTree().getPageManager(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java index 4908c08..1001af9 100644 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java +++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java @@ -34,7 +34,7 @@ public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback @Override public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath) throws HyracksDataException { - if (diskComponentFilePath.endsWith(LSMRTreeFileManager.RTREE_STRING)) { + if (diskComponentFilePath.endsWith(LSMRTreeFileManager.RTREE_SUFFIX)) { LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) diskComponent; IMetadataPageManager metadataPageManager = (IMetadataPageManager) rtreeComponent.getRTree().getPageManager(); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java index f4b638f..49f5457 100644 --- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java +++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java @@ -18,37 +18,34 @@ */ package org.apache.asterix.common.transactions; -import org.apache.asterix.common.api.ThreadExecutor; -import org.apache.asterix.common.api.INcApplicationContext; import org.apache.asterix.common.api.IDatasetLifecycleManager; +import org.apache.asterix.common.api.INcApplicationContext; +import org.apache.asterix.common.api.ThreadExecutor; import org.apache.hyracks.api.io.IIOManager; import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker; import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public interface IAppRuntimeContextProvider { - public ThreadExecutor getThreadExecutor(); - - public IBufferCache getBufferCache(); + ThreadExecutor getThreadExecutor(); - public IFileMapProvider getFileMapManager(); + IBufferCache getBufferCache(); - public ITransactionSubsystem getTransactionSubsystem(); + ITransactionSubsystem getTransactionSubsystem(); - public IDatasetLifecycleManager getDatasetLifecycleManager(); + IDatasetLifecycleManager getDatasetLifecycleManager(); - public double getBloomFilterFalsePositiveRate(); + double getBloomFilterFalsePositiveRate(); - public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID); + ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID); - public ILSMIOOperationScheduler getLSMIOScheduler(); + ILSMIOOperationScheduler getLSMIOScheduler(); - public ILocalResourceRepository getLocalResourceRepository(); + ILocalResourceRepository getLocalResourceRepository(); - public IIOManager getIOManager(); + IIOManager getIOManager(); - public INcApplicationContext getAppContext(); + INcApplicationContext getAppContext(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java index 8dbd98f..a8b15d2 100644 --- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java +++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java @@ -54,6 +54,7 @@ public class LSMComponentProperties { job.getLSMIndexOperationContext()); opType = job.getLSMOpType(); } + public LSMComponentProperties() { } @@ -132,7 +133,7 @@ public class LSMComponentProperties { componentId.append(File.separator); componentId.append(idxName); componentId.append(File.separator); - componentId.append(fileName.substring(0, fileName.lastIndexOf(AbstractLSMIndexFileManager.SPLIT_STRING))); + componentId.append(fileName.substring(0, fileName.lastIndexOf(AbstractLSMIndexFileManager.DELIMITER))); return componentId.toString(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java index ebebfbd..177bed2 100644 --- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java +++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java @@ -26,7 +26,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProv import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.IStorageManager; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; import org.apache.hyracks.storage.common.file.IResourceIdFactory; public class RuntimeComponentsProvider implements IStorageManager, ILSMIOOperationSchedulerProvider { @@ -49,11 +48,6 @@ public class RuntimeComponentsProvider implements IStorageManager, ILSMIOOperati } @Override - public IFileMapProvider getFileMapProvider(INCServiceContext ctx) { - return ((INcApplicationContext) ctx.getApplicationContext()).getFileMapManager(); - } - - @Override public ILocalResourceRepository getLocalResourceRepository(INCServiceContext ctx) { return ((INcApplicationContext) ctx.getApplicationContext()).getLocalResourceRepository(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java index ba34414..a27f987 100644 --- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java +++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java @@ -22,9 +22,9 @@ import static org.mockito.Mockito.mock; import java.util.concurrent.Executors; -import org.apache.asterix.common.api.ThreadExecutor; -import org.apache.asterix.common.api.INcApplicationContext; import org.apache.asterix.common.api.IDatasetLifecycleManager; +import org.apache.asterix.common.api.INcApplicationContext; +import org.apache.asterix.common.api.ThreadExecutor; import org.apache.asterix.common.transactions.IAppRuntimeContextProvider; import org.apache.asterix.common.transactions.ITransactionSubsystem; import org.apache.hyracks.api.io.IIOManager; @@ -32,7 +32,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler; import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker; import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; class TestRuntimeContextProvider implements IAppRuntimeContextProvider { @@ -50,11 +49,6 @@ class TestRuntimeContextProvider implements IAppRuntimeContextProvider { } @Override - public IFileMapProvider getFileMapManager() { - throw new UnsupportedOperationException(); - } - - @Override public ITransactionSubsystem getTransactionSubsystem() { throw new UnsupportedOperationException(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/pom.xml ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml index ddba2d8..f5565fb 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml +++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml @@ -102,5 +102,9 @@ org.apache.commons commons-collections4 + + commons-io + commons-io + http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java index 8f36fcd..8a91547 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java @@ -63,7 +63,7 @@ public class ErrorCode { public static final int FILE_IS_NOT_DIRECTORY = 27; public static final int CANNOT_READ_FILE = 28; public static final int UNIDENTIFIED_IO_ERROR_READING_FILE = 29; - public static final int FILE_DOES_NOT_EXISTS = 30; + public static final int FILE_DOES_NOT_EXIST = 30; public static final int UNIDENTIFIED_IO_ERROR_DELETING_DIR = 31; public static final int RESULT_NO_RECORD = 32; public static final int DUPLICATE_KEY = 33; @@ -100,6 +100,22 @@ public class ErrorCode { public static final int FLUSH_ON_CLOSED_WRITER = 64; public static final int FAIL_ON_FAILED_WRITER = 65; public static final int MISSED_FAIL_CALL = 66; + public static final int CANNOT_CREATE_FILE = 67; + public static final int NO_MAPPING_FOR_FILE_ID = 68; + public static final int NO_MAPPING_FOR_FILENAME = 69; + public static final int CANNOT_GET_NUMBER_OF_ELEMENT_FROM_INACTIVE_FILTER = 70; + public static final int CANNOT_CREATE_BLOOM_FILTER_BUILDER_FOR_INACTIVE_FILTER = 71; + public static final int CANNOT_CREATE_BLOOM_FILTER_WITH_NUMBER_OF_PAGES = 72; + public static final int CANNOT_ADD_TUPLES_TO_DUMMY_BLOOM_FILTER = 73; + public static final int CANNOT_CREATE_ACTIVE_BLOOM_FILTER = 74; + public static final int CANNOT_DEACTIVATE_INACTIVE_BLOOM_FILTER = 75; + public static final int CANNOT_DESTROY_ACTIVE_BLOOM_FILTER = 76; + public static final int CANNOT_PURGE_ACTIVE_INDEX = 77; + public static final int CANNOT_PURGE_ACTIVE_BLOOM_FILTER = 78; + public static final int CANNOT_BULK_LOAD_NON_EMPTY_TREE = 79; + public static final int CANNOT_CREATE_EXISTING_INDEX = 80; + public static final int FILE_ALREADY_MAPPED = 81; + public static final int FILE_ALREADY_EXISTS = 82; // Compilation error codes. public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000; http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java index 12a138a..b48ba14 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java @@ -56,12 +56,12 @@ public final class FileReference implements Serializable { if (!(o instanceof FileReference)) { return false; } - return file.equals(((FileReference) o).file); + return path.equals(((FileReference) o).path) && dev.equals(((FileReference) o).dev); } @Override public int hashCode() { - return file.hashCode(); + return path.hashCode(); } /** http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileHandle.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileHandle.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileHandle.java index a24c667..768ab87 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileHandle.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileHandle.java @@ -18,9 +18,8 @@ */ package org.apache.hyracks.api.io; -/** - * IFileHandle is an interface that exists only for Java compilers to perform static typing - * when handling file handle objects. Users must not implement this interface. - */ +@FunctionalInterface public interface IFileHandle { + + FileReference getFileReference(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java index 50ef4f0..b0cc07a 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java @@ -59,7 +59,7 @@ public interface IIOManager { public long getSize(IFileHandle fileHandle); - public void deleteWorkspaceFiles(); + public void deleteWorkspaceFiles() throws HyracksDataException; /** * @param ioDeviceId @@ -79,6 +79,7 @@ public interface IIOManager { /** * Gets a file reference from an absolute path + * * @deprecated * use getFileRef(int ioDeviceId, String path) instead * @param path @@ -90,6 +91,7 @@ public interface IIOManager { /** * Create a workspace file with the given prefix + * * @param prefix * @return A FileReference for the created workspace file * @throws HyracksDataException http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java new file mode 100644 index 0000000..f4c5114 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java @@ -0,0 +1,92 @@ +/* + * 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.api.util; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; + +import org.apache.commons.io.FileUtils; +import org.apache.hyracks.api.exceptions.ErrorCode; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.io.FileReference; + +/** + * This util class takes care of creation and deletion of files and directories + * and throws the appropriate error in case of failure. + */ +public class IoUtil { + + private IoUtil() { + } + + /** + * Delete a file + * + * @param fileRef + * the file to be deleted + * @throws HyracksDataException + * if the file doesn't exist or if it couldn't be deleted + */ + public static void delete(FileReference fileRef) throws HyracksDataException { + delete(fileRef.getFile()); + } + + /** + * Delete a file + * + * @param file + * the file to be deleted + * @throws HyracksDataException + * if the file doesn't exist or if it couldn't be deleted + */ + public static void delete(File file) throws HyracksDataException { + try { + if (file.isDirectory()) { + FileUtils.deleteDirectory(file); + } else { + Files.delete(file.toPath()); + } + } catch (IOException e) { + throw HyracksDataException.create(ErrorCode.CANNOT_DELETE_FILE, e, file.getAbsolutePath()); + } + } + + /** + * Create a file on disk + * + * @param fileRef + * the file to create + * @throws HyracksDataException + * if the file already exists or if it couldn't be created + */ + public static void create(FileReference fileRef) throws HyracksDataException { + if (fileRef.getFile().exists()) { + throw HyracksDataException.create(ErrorCode.FILE_ALREADY_EXISTS, fileRef.getAbsolutePath()); + } + fileRef.getFile().getParentFile().mkdirs(); + try { + if (!fileRef.getFile().createNewFile()) { + throw HyracksDataException.create(ErrorCode.FILE_ALREADY_EXISTS, fileRef.getAbsolutePath()); + } + } catch (IOException e) { + throw HyracksDataException.create(ErrorCode.CANNOT_CREATE_FILE, e, fileRef.getAbsolutePath()); + } + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties index 4bf069c..9f983a7 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties @@ -46,7 +46,7 @@ 27 = File %1$s is not a directory 28 = User doesn't have read permissions on the file %1$s 29 = Unidentified IO error occurred while reading the file %1$s -30 = File %1$s doesn't exists +30 = File %1$s doesn't exist 31 = Unidentified IO error occurred while deleting the dir %1$s 32 = No record for partition %1$s of result set %2$s 33 = Inserting duplicate keys into the primary storage @@ -83,5 +83,21 @@ 64 = Data pipeline protocol violation: flush() is called on a closed writer 65 = Data pipeline protocol violation: fail() is called twice on a writer 66 = Data pipeline protocol violation: fail() is not called by the upstream when there is a failure in the downstream +67 = Cannot create the file: %1$s +68 = No mapping found for file id %1$s +69 = No mapping found for filename %1$s +70 = Cannot get the number of elements from inactive bloom filter +71 = Failed to create the bloom filter builder since it is not active +72 = Cannot create a bloom filter with number of pages = %1$s +73 = Cannot add elements to this filter since it is supposed to be empty (number of elements hint passed to the filter during construction was 0) +74 = Failed to create the bloom filter since it is active +75 = Failed to deactivate the bloom filter since it is inactive +76 = Failed to destroy the bloom filter since it is active +77 = Failed to purge the index since it is active +78 = Failed to purge the bloom filter since it is active +79 = Cannot bulk-load a non-empty tree +80 = Cannot create index because it already exists +81 = File %1$s is already mapped +82 = Failed to create the file %1$s because it already exists 10000 = The given rule collection %1$s is not an instance of the List class. http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/FileHandle.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/FileHandle.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/FileHandle.java index 594b2c3..f71dcdf 100644 --- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/FileHandle.java +++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/FileHandle.java @@ -22,6 +22,8 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; +import org.apache.hyracks.api.exceptions.ErrorCode; +import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.api.io.IFileHandle; import org.apache.hyracks.api.io.IIOManager; @@ -37,15 +39,23 @@ public class FileHandle implements IFileHandle { this.fileRef = fileRef; } + /** + * Open the file + * + * @param rwMode + * @param syncMode + * @throws IOException + */ public void open(IIOManager.FileReadWriteMode rwMode, IIOManager.FileSyncMode syncMode) throws IOException { String mode; + if (!fileRef.getFile().exists()) { + throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileRef.getAbsolutePath()); + } switch (rwMode) { case READ_ONLY: mode = "r"; break; - case READ_WRITE: - fileRef.getFile().getAbsoluteFile().getParentFile().mkdirs(); switch (syncMode) { case METADATA_ASYNC_DATA_ASYNC: mode = "rw"; @@ -78,6 +88,7 @@ public class FileHandle implements IFileHandle { raf = null; } + @Override public FileReference getFileReference() { return fileRef; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java index 722ff9e..92831f4 100644 --- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java +++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java @@ -38,6 +38,7 @@ import org.apache.hyracks.api.io.IFileHandle; import org.apache.hyracks.api.io.IIOFuture; import org.apache.hyracks.api.io.IIOManager; import org.apache.hyracks.api.io.IODeviceHandle; +import org.apache.hyracks.api.util.IoUtil; public class IOManager implements IIOManager { /* @@ -130,8 +131,7 @@ public class IOManager implements IIOManager { while (remaining > 0) { int len = ((FileHandle) fHandle).getFileChannel().write(data, offset); if (len < 0) { - throw new HyracksDataException( - "Error writing to file: " + ((FileHandle) fHandle).getFileReference().toString()); + throw new HyracksDataException("Error writing to file: " + fHandle.getFileReference().toString()); } remaining -= len; offset += len; @@ -164,8 +164,7 @@ public class IOManager implements IIOManager { len = fileChannel.write(dataArray); } if (len < 0) { - throw new HyracksDataException( - "Error writing to file: " + ((FileHandle) fHandle).getFileReference().toString()); + throw new HyracksDataException("Error writing to file: " + fHandle.getFileReference().toString()); } remaining -= len; offset += len; @@ -333,17 +332,17 @@ public class IOManager implements IIOManager { @Override public long getSize(IFileHandle fileHandle) { - return ((FileHandle) fileHandle).getFileReference().getFile().length(); + return fileHandle.getFileReference().getFile().length(); } @Override - public void deleteWorkspaceFiles() { + public void deleteWorkspaceFiles() throws HyracksDataException { for (IODeviceHandle ioDevice : workspaces) { File workspaceFolder = new File(ioDevice.getMount(), ioDevice.getWorkspace()); if (workspaceFolder.exists() && workspaceFolder.isDirectory()) { File[] workspaceFiles = workspaceFolder.listFiles(WORKSPACE_FILES_FILTER); for (File workspaceFile : workspaceFiles) { - workspaceFile.delete(); + IoUtil.delete(workspaceFile); } } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java index 73ac7a8..3d33ca1 100644 --- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java +++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java @@ -25,7 +25,6 @@ import org.apache.hyracks.storage.common.ILocalResourceRepository; import org.apache.hyracks.storage.common.IResourceLifecycleManager; import org.apache.hyracks.storage.common.IStorageManager; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; import org.apache.hyracks.storage.common.file.ResourceIdFactory; public class BTreeHelperStorageManager implements IStorageManager { @@ -42,11 +41,6 @@ public class BTreeHelperStorageManager implements IStorageManager { } @Override - public IFileMapProvider getFileMapProvider(INCServiceContext ctx) { - return RuntimeContext.get(ctx).getFileMapManager(); - } - - @Override public ILocalResourceRepository getLocalResourceRepository(INCServiceContext ctx) { return RuntimeContext.get(ctx).getLocalResourceRepository(); } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java index a2175ee..32f8418 100644 --- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java +++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java @@ -39,7 +39,7 @@ import org.apache.hyracks.storage.common.file.IFileMapProvider; import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory; import org.apache.hyracks.storage.common.file.ResourceIdFactory; import org.apache.hyracks.storage.common.file.ResourceIdFactoryProvider; -import org.apache.hyracks.storage.common.file.TransientFileMapManager; +import org.apache.hyracks.storage.common.file.FileMapManager; import org.apache.hyracks.storage.common.file.TransientLocalResourceRepositoryFactory; public class RuntimeContext { @@ -56,7 +56,7 @@ public class RuntimeContext { }; public RuntimeContext(INCServiceContext appCtx) throws HyracksDataException { - fileMapManager = new TransientFileMapManager(); + fileMapManager = new FileMapManager(); ICacheMemoryAllocator allocator = new HeapBufferAllocator(); IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, 32768, 50); bufferCache = new BufferCache(appCtx.getIoManager(), prs, new DelayPageCleanerPolicy(1000), fileMapManager, 100, http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java ---------------------------------------------------------------------- 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 3b49d08..47a9734 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 @@ -21,6 +21,7 @@ package org.apache.hyracks.storage.am.bloomfilter.impls; import java.nio.ByteBuffer; +import org.apache.hyracks.api.exceptions.ErrorCode; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference; @@ -29,7 +30,6 @@ 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.file.BufferedFileHandle; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public class BloomFilter { @@ -40,7 +40,6 @@ public class BloomFilter { private static final int NUM_BITS_OFFSET = NUM_ELEMENTS_OFFSET + 8; // 12 private final IBufferCache bufferCache; - private final IFileMapProvider fileMapProvider; private final FileReference file; private final int[] keyFields; private int fileId = -1; @@ -51,13 +50,11 @@ public class BloomFilter { private long numElements; private long numBits; private final int numBitsPerPage; - private final static byte[] ZERO_BUFFER = new byte[131072]; // 128kb - private final static long SEED = 0L; + private static final byte[] ZERO_BUFFER = new byte[131072]; // 128kb + private static final long SEED = 0L; - public BloomFilter(IBufferCache bufferCache, IFileMapProvider fileMapProvider, FileReference file, int[] keyFields) - throws HyracksDataException { + public BloomFilter(IBufferCache bufferCache, FileReference file, int[] keyFields) throws HyracksDataException { this.bufferCache = bufferCache; - this.fileMapProvider = fileMapProvider; this.file = file; this.keyFields = keyFields; this.numBitsPerPage = bufferCache.getPageSize() * Byte.SIZE; @@ -80,7 +77,7 @@ public class BloomFilter { public long getNumElements() throws HyracksDataException { if (!isActivated) { - throw new HyracksDataException("The bloom filter is not activated."); + throw HyracksDataException.create(ErrorCode.CANNOT_GET_NUMBER_OF_ELEMENT_FROM_INACTIVE_FILTER); } return numElements; } @@ -94,8 +91,8 @@ public class BloomFilter { long hash = Math.abs((hashes[0] + i * hashes[1]) % numBits); // we increment the page id by one, since the metadata page id of the filter is 0. - ICachedPage page = bufferCache.pin( - BufferedFileHandle.getDiskPageId(fileId, (int) (hash / numBitsPerPage) + 1), false); + ICachedPage page = + bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, (int) (hash / numBitsPerPage) + 1), false); page.acquireReadLatch(); try { ByteBuffer buffer = page.getBuffer(); @@ -115,40 +112,22 @@ public class BloomFilter { return true; } - private void prepareFile() throws HyracksDataException { - boolean fileIsMapped = false; - synchronized (fileMapProvider) { - fileIsMapped = fileMapProvider.isMapped(file); - if (!fileIsMapped) { - bufferCache.createFile(file); - } - fileId = fileMapProvider.lookupFileId(file); - try { - // Also creates the file if it doesn't exist yet. - bufferCache.openFile(fileId); - } catch (HyracksDataException e) { - // Revert state of buffer cache since file failed to open. - if (!fileIsMapped) { - bufferCache.deleteFile(fileId, false); - } - throw e; - } - } - } - public synchronized void create() throws HyracksDataException { if (isActivated) { - throw new HyracksDataException("Failed to create the bloom filter since it is activated."); + throw HyracksDataException.create(ErrorCode.CANNOT_CREATE_ACTIVE_BLOOM_FILTER); } - prepareFile(); - bufferCache.closeFile(fileId); + fileId = bufferCache.createFile(file); } public synchronized void activate() throws HyracksDataException { if (isActivated) { return; } - prepareFile(); + if (fileId >= 0) { + bufferCache.openFile(fileId); + } else { + fileId = bufferCache.openFile(file); + } readBloomFilterMetaData(); isActivated = true; } @@ -176,23 +155,26 @@ public class BloomFilter { public synchronized void deactivate() throws HyracksDataException { if (!isActivated) { - return; + throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_BLOOM_FILTER); } bufferCache.closeFile(fileId); isActivated = false; } - public synchronized void destroy() throws HyracksDataException { + public void purge() throws HyracksDataException { if (isActivated) { - throw new HyracksDataException("Failed to destroy the bloom filter since it is activated."); + throw HyracksDataException.create(ErrorCode.CANNOT_PURGE_ACTIVE_BLOOM_FILTER); } + bufferCache.purgeHandle(fileId); + // after purging, the fileId has no mapping and no meaning + fileId = -1; + } - file.delete(); - if (fileId == -1) { - return; + public synchronized void destroy() throws HyracksDataException { + if (isActivated) { + throw HyracksDataException.create(ErrorCode.CANNOT_DESTROY_ACTIVE_BLOOM_FILTER); } - bufferCache.deleteFile(fileId, false); - fileId = -1; + bufferCache.deleteFile(file); } public IIndexBulkLoader createBuilder(long numElements, int numHashes, int numBitsPerElement) @@ -212,7 +194,7 @@ public class BloomFilter { public BloomFilterBuilder(long numElements, int numHashes, int numBitsPerElement) throws HyracksDataException { if (!isActivated) { - throw new HyracksDataException("Failed to create the bloom filter builder since it is not activated."); + throw HyracksDataException.create(ErrorCode.CANNOT_CREATE_BLOOM_FILTER_BUILDER_FOR_INACTIVE_FILTER); } queue = bufferCache.createFIFOQueue(); this.numElements = numElements; @@ -220,7 +202,7 @@ public class BloomFilter { numBits = this.numElements * numBitsPerElement; long tmp = (long) Math.ceil(numBits / (double) numBitsPerPage); if (tmp > Integer.MAX_VALUE) { - throw new HyracksDataException("Cannot create a bloom filter with his huge number of pages."); + throw HyracksDataException.create(ErrorCode.CANNOT_CREATE_BLOOM_FILTER_WITH_NUMBER_OF_PAGES, tmp); } numPages = (int) tmp; pages = new ICachedPage[numPages]; @@ -259,13 +241,12 @@ public class BloomFilter { @Override public void add(ITupleReference tuple) throws HyracksDataException { if (numPages == 0) { - throw new HyracksDataException( - "Cannot add elements to this filter since it is supposed to be empty (number of elements hint passed to the filter during construction was 0)."); + throw HyracksDataException.create(ErrorCode.CANNOT_ADD_TUPLES_TO_DUMMY_BLOOM_FILTER); } MurmurHash128Bit.hash3_x64_128(tuple, keyFields, SEED, hashes); for (int i = 0; i < numHashes; ++i) { long hash = Math.abs((hashes[0] + i * hashes[1]) % numBits); - ICachedPage page = pages[((int) (hash / numBitsPerPage))]; + ICachedPage page = pages[(int) (hash / numBitsPerPage)]; ByteBuffer buffer = page.getBuffer(); int byteIndex = (int) (hash % numBitsPerPage) >> 3; // divide by 8 byte b = buffer.get(byteIndex); @@ -297,8 +278,8 @@ public class BloomFilter { bufferCache.returnPage(p, false); } } - if (metaDataPage != null ){ - bufferCache.returnPage(metaDataPage,false); + if (metaDataPage != null) { + bufferCache.returnPage(metaDataPage, false); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java index c3a1718..8d01bb7 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java @@ -22,21 +22,18 @@ package org.apache.hyracks.storage.am.bloomfilter.impls; import org.apache.hyracks.api.exceptions.HyracksDataException; import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public class BloomFilterFactory { private final IBufferCache bufferCache; - private final IFileMapProvider fileMapProvider; private final int[] bloomFilterKeyFields; - public BloomFilterFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider, int[] bloomFilterKeyFields) { + public BloomFilterFactory(IBufferCache bufferCache, int[] bloomFilterKeyFields) { this.bufferCache = bufferCache; - this.fileMapProvider = fileMapProvider; this.bloomFilterKeyFields = bloomFilterKeyFields; } public BloomFilter createBloomFiltertInstance(FileReference file) throws HyracksDataException { - return new BloomFilter(bufferCache, fileMapProvider, file, bloomFilterKeyFields); + return new BloomFilter(bufferCache, file, bloomFilterKeyFields); } public int[] getBloomFilterKeyFields() { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java index 90bc126..24d2a37 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java @@ -55,9 +55,8 @@ public class BTreeResource implements IResource { IBufferCache bufferCache = storageManager.getBufferCache(ctx); IIOManager ioManager = ctx.getIoManager(); FileReference resourceRef = ioManager.resolve(path); - return BTreeUtils.createBTree(bufferCache, storageManager.getFileMapProvider(ctx), typeTraits, - comparatorFactories, BTreeLeafFrameType.REGULAR_NSM, resourceRef, - pageManagerFactory.createPageManager(bufferCache)); + return BTreeUtils.createBTree(bufferCache, typeTraits, comparatorFactories, BTreeLeafFrameType.REGULAR_NSM, + resourceRef, pageManagerFactory.createPageManager(bufferCache)); } @Override http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java ---------------------------------------------------------------------- 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 7f57524..ce35fbf 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 @@ -63,7 +63,6 @@ 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.file.BufferedFileHandle; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public class BTree extends AbstractTreeIndex { @@ -77,11 +76,10 @@ public class BTree extends AbstractTreeIndex { private final ReadWriteLock treeLatch; private final int maxTupleSize; - public BTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IPageManager freePageManager, - ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory, - IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) { - super(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, - fieldCount, file); + public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory, + ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount, + FileReference file) { + super(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file); this.treeLatch = new ReentrantReadWriteLock(true); this.smoCounter = new AtomicInteger(); ITreeIndexFrame leafFrame = leafFrameFactory.createFrame(); @@ -96,11 +94,11 @@ public class BTree extends AbstractTreeIndex { RangePredicate diskOrderScanPred = new RangePredicate(null, null, true, true, ctx.getCmp(), ctx.getCmp()); int maxPageId = freePageManager.getMaxPageId(ctx.getMetaFrame()); int currentPageId = bulkloadLeafStart; - ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false); + ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), currentPageId), false); page.acquireReadLatch(); try { cursor.setBufferCache(bufferCache); - cursor.setFileId(fileId); + cursor.setFileId(getFileId()); cursor.setCurrentPageId(currentPageId); cursor.setMaxPageId(maxPageId); ctx.getCursorInitialState().setPage(page); @@ -129,7 +127,7 @@ public class BTree extends AbstractTreeIndex { } private void validate(BTreeOpContext ctx, int pageId) throws HyracksDataException { - ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); ctx.getInteriorFrame().setPage(page); PageValidationInfo currentPvi = ctx.getValidationInfos().peekFirst(); @@ -206,14 +204,14 @@ public class BTree extends AbstractTreeIndex { repeatOp = false; } cursor.setBufferCache(bufferCache); - cursor.setFileId(fileId); + cursor.setFileId(getFileId()); } private void unsetSmPages(BTreeOpContext ctx) throws HyracksDataException { ICachedPage originalPage = ctx.getInteriorFrame().getPage(); for (int i = 0; i < ctx.getSmPages().size(); i++) { int pageId = ctx.getSmPages().get(i); - ICachedPage smPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage smPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); smPage.acquireWriteLatch(); try { ctx.getInteriorFrame().setPage(smPage); @@ -238,11 +236,11 @@ public class BTree extends AbstractTreeIndex { private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException { // Make sure the root is always in the same page. ICachedPage leftNode = - bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.getSplitKey().getLeftPage()), false); + bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), ctx.getSplitKey().getLeftPage()), false); leftNode.acquireWriteLatch(); try { int newLeftId = freePageManager.takePage(ctx.getMetaFrame()); - ICachedPage newLeftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, newLeftId), true); + ICachedPage newLeftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), newLeftId), true); newLeftNode.acquireWriteLatch(); try { boolean largePage = false; @@ -350,7 +348,7 @@ public class BTree extends AbstractTreeIndex { } } int rightPageId = freePageManager.takePage(ctx.getMetaFrame()); - ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true); + ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), rightPageId), true); rightNode.acquireWriteLatch(); try { IBTreeLeafFrame rightFrame = ctx.createLeafFrame(); @@ -466,7 +464,8 @@ public class BTree extends AbstractTreeIndex { switch (spaceStatus) { case INSUFFICIENT_SPACE: { int rightPageId = freePageManager.takePage(ctx.getMetaFrame()); - ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true); + ICachedPage rightNode = + bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), rightPageId), true); rightNode.acquireWriteLatch(); try { IBTreeFrame rightFrame = ctx.createInteriorFrame(); @@ -545,7 +544,7 @@ public class BTree extends AbstractTreeIndex { } private ICachedPage isConsistent(int pageId, BTreeOpContext ctx) throws Exception { - ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); node.acquireReadLatch(); ctx.getInteriorFrame().setPage(node); boolean isConsistent = ctx.getPageLsns().getLast() == ctx.getInteriorFrame().getPageLsn(); @@ -559,7 +558,7 @@ public class BTree extends AbstractTreeIndex { private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatched, BTreeOpContext ctx) throws HyracksDataException { - ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); ctx.getInteriorFrame().setPage(node); // this check performs an unprotected read in the page // the following could happen: TODO fill out @@ -620,8 +619,8 @@ public class BTree extends AbstractTreeIndex { case UPDATE: { // Is there a propagated split key? if (ctx.getSplitKey().getBuffer() != null) { - ICachedPage interiorNode = - bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage interiorNode = bufferCache + .pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); interiorNode.acquireWriteLatch(); try { // Insert or update op. Both can cause split keys to propagate upwards. @@ -769,7 +768,7 @@ public class BTree extends AbstractTreeIndex { public void printTree(int pageId, ICachedPage parent, boolean unpin, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, byte treeHeight, ISerializerDeserializer[] keySerdes, StringBuilder strBuilder, MultiComparator cmp) throws Exception { - ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false); + ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), pageId), false); node.acquireReadLatch(); try { if (parent != null && unpin == true) { @@ -1037,7 +1036,7 @@ public class BTree extends AbstractTreeIndex { splitKey.setRightPage(leafFrontier.pageId); } if (tupleSize > maxTupleSize) { - final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId); + final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId); // calculate required number of pages. int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize()); final int multiplier = @@ -1052,7 +1051,7 @@ public class BTree extends AbstractTreeIndex { leafFrame.initBuffer((byte) 0); ((IBTreeLeafFrame) leafFrame).setLargeFlag(true); } else { - final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId); + final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId); leafFrontier.page = bufferCache.confiscatePage(dpid); leafFrame.setPage(leafFrontier.page); leafFrame.initBuffer((byte) 0); @@ -1116,7 +1115,7 @@ public class BTree extends AbstractTreeIndex { ((IBTreeInteriorFrame) interiorFrame).deleteGreatest(); int finalPageId = freePageManager.takePage(metaFrame); - bufferCache.setPageDiskId(frontier.page, BufferedFileHandle.getDiskPageId(fileId, finalPageId)); + bufferCache.setPageDiskId(frontier.page, BufferedFileHandle.getDiskPageId(getFileId(), finalPageId)); pagesToWrite.add(frontier.page); splitKey.setLeftPage(finalPageId); @@ -1176,7 +1175,7 @@ public class BTree extends AbstractTreeIndex { } private void setPageDpid(ICachedPage page, int pageId) { - bufferCache.setPageDiskId(page, BufferedFileHandle.getDiskPageId(fileId, pageId)); + bufferCache.setPageDiskId(page, BufferedFileHandle.getDiskPageId(getFileId(), pageId)); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ae3daf6e/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java ---------------------------------------------------------------------- diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java index 5cc9599..303777e 100644 --- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java +++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java @@ -35,29 +35,26 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory; import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory; import org.apache.hyracks.storage.common.MultiComparator; import org.apache.hyracks.storage.common.buffercache.IBufferCache; -import org.apache.hyracks.storage.common.file.IFileMapProvider; public class BTreeUtils { - public static BTree createBTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, - ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, - FileReference file, IPageManager freePageManager) throws HyracksDataException { + public static BTree createBTree(IBufferCache bufferCache, ITypeTraits[] typeTraits, + IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, FileReference file, + IPageManager freePageManager) throws HyracksDataException { TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits); ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType); ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory); - BTree btree = new BTree(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, - cmpFactories, typeTraits.length, file); - return btree; + return new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, + typeTraits.length, file); } - public static BTree createBTree(IBufferCache bufferCache, IPageManager freePageManager, - IFileMapProvider fileMapProvider, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, - BTreeLeafFrameType leafType, FileReference file) throws HyracksDataException { + public static BTree createBTree(IBufferCache bufferCache, IPageManager freePageManager, ITypeTraits[] typeTraits, + IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, FileReference file) + throws HyracksDataException { TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits); ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType); ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory); - BTree btree = new BTree(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, - cmpFactories, typeTraits.length, file); - return btree; + return new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, + typeTraits.length, file); } // Creates a new MultiComparator by constructing new IBinaryComparators.