blob: 1a40607ac8456d73c8145a461fb326d50da59412 [file] [log] [blame]
/**
* 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.apex.malhar.lib.fileaccess;
import java.io.IOException;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.io.file.tfile.DTFile;
import org.apache.hadoop.io.file.tfile.DTFile.Reader;
import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner.Entry;
import org.apache.hadoop.io.file.tfile.TFile;
import com.datatorrent.netlet.util.Slice;
/**
* {@link DTFile} wrapper for HDSFileReader
* <br>
* {@link DTFile} has exact same format as {@link TFile} with a much faster {@link Reader} implementation
* <br>
* DTFileReader is also fully compatible with any file generated by {@link TFileWriter}. So there is no corresponding "DTFileWriter"
*
*
* @since 2.0.0
*/
@InterfaceStability.Evolving
public class DTFileReader implements FileAccess.FileReader
{
private final Reader reader;
private final Scanner scanner;
private final FSDataInputStream fsdis;
public DTFileReader(FSDataInputStream fsdis, long fileLength, Configuration conf) throws IOException
{
this.fsdis = fsdis;
reader = new Reader(fsdis, fileLength, conf);
scanner = reader.createScanner();
}
/**
* Unlike the TFile.Reader.close method this will close the wrapped InputStream.
* @see java.io.Closeable#close()
*/
@Override
public void close() throws IOException
{
scanner.close();
reader.close();
fsdis.close();
}
@Override
public void readFully(TreeMap<Slice, Slice> data) throws IOException
{
scanner.rewind();
for (; !scanner.atEnd(); scanner.advance()) {
Entry en = scanner.entry();
Slice key = new Slice(en.getBlockBuffer(), en.getKeyOffset(), en.getKeyLength());
Slice value = new Slice(en.getBlockBuffer(), en.getValueOffset(), en.getValueLength());
data.put(key, value);
}
}
@Override
public void reset() throws IOException
{
scanner.rewind();
}
@Override
public boolean seek(Slice key) throws IOException
{
return scanner.seekTo(key.buffer, key.offset, key.length);
}
@Override
public boolean peek(Slice key, Slice value) throws IOException
{
if (scanner.atEnd()) {
return false;
}
Entry en = scanner.entry();
key.buffer = en.getBlockBuffer();
key.offset = en.getKeyOffset();
key.length = en.getKeyLength();
value.buffer = en.getBlockBuffer();
value.offset = en.getValueOffset();
value.length = en.getValueLength();
return true;
}
@Override
public boolean next(Slice key, Slice value) throws IOException
{
if (peek(key, value)) {
scanner.advance();
return true;
} else {
return false;
}
}
@Override
public boolean hasNext()
{
return !scanner.atEnd();
}
}