HADOOP-6497. Add an adapter for Avro's SeekableInput interface, so that Avro can read FileSystem data. Contributed by Aaron Kimball.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@900607 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Doug Cutting 2010-01-18 23:39:04 +00:00
parent 4cd159e83e
commit f3477eb324
3 changed files with 155 additions and 0 deletions

View File

@ -104,6 +104,10 @@ Trunk (unreleased changes)
HADOOP-6492. Make some Avro serialization APIs public. HADOOP-6492. Make some Avro serialization APIs public.
(Aaron Kimball via cutting) (Aaron Kimball via cutting)
HADOOP-6497. Add an adapter for Avro's SeekableInput interface, so
that Avro can read FileSystem data.
(Aaron Kimball via cutting)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -0,0 +1,63 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.avro.file.SeekableInput;
/** Adapts an {@link FSDataInputStream} to Avro's SeekableInput interface. */
public class AvroFSInput implements Closeable, SeekableInput {
private final FSDataInputStream stream;
private final long len;
/** Construct given an {@link FSDataInputStream} and its length. */
public AvroFSInput(final FSDataInputStream in, final long len) {
this.stream = in;
this.len = len;
}
/** Construct given a {@link FileContext} and a {@link Path}. */
public AvroFSInput(final FileContext fc, final Path p) throws IOException {
FileStatus status = fc.getFileStatus(p);
this.len = status.getLen();
this.stream = fc.open(p);
}
public long length() {
return len;
}
public int read(byte[] b, int off, int len) throws IOException {
return stream.read(b, off, len);
}
public void seek(long p) throws IOException {
stream.seek(p);
}
public long tell() throws IOException {
return stream.getPos();
}
public void close() throws IOException {
stream.close();
}
}

View File

@ -0,0 +1,88 @@
/**
* 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.hadoop.fs;
import java.io.BufferedWriter;
import java.io.IOException;
import java.io.OutputStreamWriter;
import org.apache.hadoop.conf.Configuration;
import junit.framework.TestCase;
public class TestAvroFSInput extends TestCase {
private static final String INPUT_DIR = "AvroFSInput";
private Path getInputPath() {
String dataDir = System.getProperty("test.build.data");
if (null == dataDir) {
return new Path(INPUT_DIR);
} else {
return new Path(new Path(dataDir), INPUT_DIR);
}
}
public void testAFSInput() throws Exception {
Configuration conf = new Configuration();
FileSystem fs = FileSystem.getLocal(conf);
Path dir = getInputPath();
if (!fs.exists(dir)) {
fs.mkdirs(dir);
}
Path filePath = new Path(dir, "foo");
if (fs.exists(filePath)) {
fs.delete(filePath, false);
}
FSDataOutputStream ostream = fs.create(filePath);
BufferedWriter w = new BufferedWriter(new OutputStreamWriter(ostream));
w.write("0123456789");
w.close();
// Create the stream
FileContext fc = FileContext.getFileContext(conf);
AvroFSInput avroFSIn = new AvroFSInput(fc, filePath);
assertEquals(10, avroFSIn.length());
// Check initial position
byte [] buf = new byte[1];
assertEquals(0, avroFSIn.tell());
// Check a read from that position.
avroFSIn.read(buf, 0, 1);
assertEquals(1, avroFSIn.tell());
assertEquals('0', (char)buf[0]);
// Check a seek + read
avroFSIn.seek(4);
assertEquals(4, avroFSIn.tell());
avroFSIn.read(buf, 0, 1);
assertEquals('4', (char)buf[0]);
assertEquals(5, avroFSIn.tell());
avroFSIn.close();
}
}