Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
0016  */
0017 
0018 package org.apache.spark.util.collection.unsafe.sort;
0019 
0020 import java.io.IOException;
0021 import java.util.Comparator;
0022 import java.util.PriorityQueue;
0023 
0024 final class UnsafeSorterSpillMerger {
0025 
0026   private int numRecords = 0;
0027   private final PriorityQueue<UnsafeSorterIterator> priorityQueue;
0028 
0029   UnsafeSorterSpillMerger(
0030       RecordComparator recordComparator,
0031       PrefixComparator prefixComparator,
0032       int numSpills) {
0033     Comparator<UnsafeSorterIterator> comparator = (left, right) -> {
0034       int prefixComparisonResult =
0035         prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
0036       if (prefixComparisonResult == 0) {
0037         return recordComparator.compare(
0038           left.getBaseObject(), left.getBaseOffset(), left.getRecordLength(),
0039           right.getBaseObject(), right.getBaseOffset(), right.getRecordLength());
0040       } else {
0041         return prefixComparisonResult;
0042       }
0043     };
0044     priorityQueue = new PriorityQueue<>(numSpills, comparator);
0045   }
0046 
0047   /**
0048    * Add an UnsafeSorterIterator to this merger
0049    */
0050   public void addSpillIfNotEmpty(UnsafeSorterIterator spillReader) throws IOException {
0051     if (spillReader.hasNext()) {
0052       // We only add the spillReader to the priorityQueue if it is not empty. We do this to
0053       // make sure the hasNext method of UnsafeSorterIterator returned by getSortedIterator
0054       // does not return wrong result because hasNext will return true
0055       // at least priorityQueue.size() times. If we allow n spillReaders in the
0056       // priorityQueue, we will have n extra empty records in the result of UnsafeSorterIterator.
0057       spillReader.loadNext();
0058       priorityQueue.add(spillReader);
0059       numRecords += spillReader.getNumRecords();
0060     }
0061   }
0062 
0063   public UnsafeSorterIterator getSortedIterator() throws IOException {
0064     return new UnsafeSorterIterator() {
0065 
0066       private UnsafeSorterIterator spillReader;
0067 
0068       @Override
0069       public int getNumRecords() {
0070         return numRecords;
0071       }
0072 
0073       @Override
0074       public boolean hasNext() {
0075         return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext());
0076       }
0077 
0078       @Override
0079       public void loadNext() throws IOException {
0080         if (spillReader != null) {
0081           if (spillReader.hasNext()) {
0082             spillReader.loadNext();
0083             priorityQueue.add(spillReader);
0084           }
0085         }
0086         spillReader = priorityQueue.remove();
0087       }
0088 
0089       @Override
0090       public Object getBaseObject() { return spillReader.getBaseObject(); }
0091 
0092       @Override
0093       public long getBaseOffset() { return spillReader.getBaseOffset(); }
0094 
0095       @Override
0096       public int getRecordLength() { return spillReader.getRecordLength(); }
0097 
0098       @Override
0099       public long getKeyPrefix() { return spillReader.getKeyPrefix(); }
0100     };
0101   }
0102 }