@Override public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException { page = initialState.getPage(); tupleIndex = 0; frame.setPage(page); positionToNextLeaf(false); }
protected void addLevel() throws HyracksDataException { NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference()); frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID); frontier.pageId = -1; frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount()); interiorFrame.setPage(frontier.page); interiorFrame.initBuffer((byte) nodeFrontiers.size()); nodeFrontiers.add(frontier); }
@Override public boolean doHasNext() throws HyracksDataException { if (numTuples <= currentTupleIndex) { return false; } // We don't latch pages since this code is only used by flush () before // bulk-loading the r-tree to disk and flush is not concurrent. // ICachedPage node1 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, tPointers[currentTupleIndex * 2]), false); try { leafFrame1.setPage(node1); frameTuple1.resetByTupleOffset(leafFrame1.getBuffer().array(), tPointers[currentTupleIndex * 2 + 1]); } finally { bufferCache.unpin(node1); } return true; }
private boolean positionToNextLeaf(boolean skipCurrent) throws HyracksDataException { while (frame.getLevel() != 0 || skipCurrent || frame.getTupleCount() == 0) { if (++currentPageId > maxPageId) { break; } releasePage(); ICachedPage nextPage = acquireNextPage(); page = nextPage; frame.setPage(page); tupleIndex = 0; skipCurrent = false; } if (currentPageId <= maxPageId) { return true; } else { return false; } }
private int compare(int[] tPointers, int tp1, int tp2i, int tp2j) throws HyracksDataException { int i1 = tPointers[tp1 * 2]; int j1 = tPointers[tp1 * 2 + 1]; int i2 = tp2i; int j2 = tp2j; ICachedPage node1 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i1), false); try { leafFrame1.setPage(node1); ICachedPage node2 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i2), false); try { leafFrame2.setPage(node2); frameTuple1.resetByTupleOffset(leafFrame1.getBuffer().array(), j1); frameTuple2.resetByTupleOffset(leafFrame2.getBuffer().array(), j2); return cmp.selectiveFieldCompare(frameTuple1, frameTuple2, comparatorFields); } finally { bufferCache.unpin(node2); } } finally { bufferCache.unpin(node1); } }
private void persistFrontiers(int level, int rightPage) throws HyracksDataException { if (level >= nodeFrontiers.size()) { rootPage = nodeFrontiers.get(level - 1).pageId; releasedLatches = true; return; } if (level < 1) { ICachedPage lastLeaf = nodeFrontiers.get(level).page; int lastLeafPage = nodeFrontiers.get(level).pageId; lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId)); putInQueue(lastLeaf); nodeFrontiers.get(level).page = null; persistFrontiers(level + 1, lastLeafPage); return; } NodeFrontier frontier = nodeFrontiers.get(level); interiorFrame.setPage(frontier.page); //just finalize = the layer right above the leaves has correct righthand pointers already if (rightPage < 0) { throw new HyracksDataException( "Error in index creation. Internal node appears to have no rightmost guide"); } ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage); int finalPageId = freePageManager.takePage(metaFrame); frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId)); putInQueue(frontier.page); frontier.pageId = finalPageId; persistFrontiers(level + 1, finalPageId); }
public byte getTreeHeight(ITreeIndexFrame frame) throws HyracksDataException { ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false); rootNode.acquireReadLatch(); try { frame.setPage(rootNode); return frame.getLevel(); } finally { rootNode.releaseReadLatch(); bufferCache.unpin(rootNode); } }
interiorFrame.setPage(frontier.page); pagesToWrite.add(frontier.page); lowerFrame = prevInteriorFrame; lowerFrame.setPage(frontier.page); interiorFrame.setPage(frontier.page); interiorFrame.initBuffer((byte) level); lowerFrame.setPage(frontier.page); propagateBulk(level + 1, true, pagesToWrite); leafFrame.setPage(nodeFrontiers.get(0).page);
leafFrontier.page = bufferCache .confiscatePage(BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId)); leafFrame.setPage(leafFrontier.page); leafFrame.initBuffer((byte) 0); leafFrame.setPage(leafFrontier.page); leafFrame.insert(tuple, AbstractSlotManager.GREATEST_KEY_INDICATOR); } catch (HyracksDataException e) {
@Override public boolean isEmpty(ITreeIndexFrame frame, int rootPage) throws HyracksDataException { ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false); rootNode.acquireReadLatch(); try { frame.setPage(rootNode); return frame.getLevel() == 0 && frame.getTupleCount() == 0; } finally { rootNode.releaseReadLatch(); bufferCache.unpin(rootNode); } } }
@Override public boolean isEmpty(ITreeIndexFrame frame, int rootPage) throws HyracksDataException { ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false); rootNode.acquireReadLatch(); try { frame.setPage(rootNode); return frame.getLevel() == 0 && frame.getTupleCount() == 0; } finally { rootNode.releaseReadLatch(); bufferCache.unpin(rootNode); } }
@Override public void init(ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory) throws HyracksDataException { currentPageId.set(1); ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, 0), true); page.acquireWriteLatch(); page.releaseWriteLatch(false); bufferCache.unpin(page); page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId.get()), true); if (leafFrameFactory != null) { page.acquireWriteLatch(); ITreeIndexFrame leafFrame = leafFrameFactory.createFrame(); leafFrame.setPage(page); leafFrame.initBuffer((byte) 0); page.releaseWriteLatch(true); } bufferCache.unpin(page); }
public AbstractTreeIndexBulkLoader(float fillFactor) throws HyracksDataException { leafFrame = leafFrameFactory.createFrame(); interiorFrame = interiorFrameFactory.createFrame(); metaFrame = freePageManager.createMetadataFrame(); queue = bufferCache.createFIFOQueue(); if (!isEmptyTree(leafFrame)) { throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE); } this.cmp = MultiComparator.create(cmpFactories); leafFrame.setMultiComparator(cmp); interiorFrame.setMultiComparator(cmp); tupleWriter = leafFrame.getTupleWriter(); NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference()); leafFrontier.pageId = freePageManager.takePage(metaFrame); leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId)); interiorFrame.setPage(leafFrontier.page); interiorFrame.initBuffer((byte) 0); interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor); leafFrame.setPage(leafFrontier.page); leafFrame.initBuffer((byte) 0); leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor); slotSize = leafFrame.getSlotSize(); nodeFrontiers.add(leafFrontier); pagesToWrite = new ArrayList<>(); compressedPageWriter = bufferCache.getCompressedPageWriter(fileId); }
page.acquireReadLatch(); try { frame.setPage(page); byte level = frame.getLevel(); while (level >= pagesByLevel.size()) {
leafFrontier.page = bufferCache.confiscatePage(dpid); leafFrame.setPage(leafFrontier.page); leafFrame.initBuffer((byte) 0); ((IBTreeLeafFrame) leafFrame).setLargeFlag(true); final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId); leafFrontier.page = bufferCache.confiscatePage(dpid); leafFrame.setPage(leafFrontier.page); leafFrame.initBuffer((byte) 0);
try { ITreeIndexFrame leafFrame = leafFrameFactory.createFrame(); leafFrame.setPage(rootNode); leafFrame.initBuffer((byte) 0); } finally {