| /* |
| * 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.pig.impl.io; |
| |
| import java.io.IOException; |
| import java.rmi.server.UID; |
| import java.security.MessageDigest; |
| import java.security.NoSuchAlgorithmException; |
| |
| import org.apache.hadoop.fs.FSDataOutputStream; |
| import org.apache.hadoop.io.WritableComparable; |
| import org.apache.hadoop.mapreduce.RecordWriter; |
| import org.apache.hadoop.mapreduce.TaskAttemptContext; |
| import org.apache.hadoop.util.Time; |
| import org.apache.pig.data.InterSedes; |
| import org.apache.pig.data.InterSedesFactory; |
| import org.apache.pig.data.Tuple; |
| |
| |
| /** |
| * A record reader used to write data compatible with {@link InterRecordWriter} |
| * It uses the default InterSedes object for serialization. |
| */ |
| public class InterRecordWriter extends |
| RecordWriter<org.apache.hadoop.io.WritableComparable, Tuple> { |
| |
| private static InterSedes sedes = InterSedesFactory.getInterSedesInstance(); |
| |
| private byte[] syncMarker; |
| private long lastSyncPos = -1; |
| private long syncMarkerInterval; |
| /** |
| * the outputstream to write out on |
| */ |
| private FSDataOutputStream out; |
| |
| /** |
| * |
| */ |
| public InterRecordWriter(FSDataOutputStream out, int syncMarkerLength, long syncMarkerInterval) { |
| this.out = out; |
| this.syncMarkerInterval = syncMarkerInterval; |
| syncMarker = new byte[syncMarkerLength]; |
| |
| try { |
| MessageDigest digester = MessageDigest.getInstance("MD5"); |
| long time = Time.now(); |
| digester.update((new UID()+"@"+time).getBytes()); |
| byte[] generatedMarker = digester.digest(); |
| System.arraycopy(generatedMarker, 0, syncMarker, 0, syncMarkerLength); |
| } catch (NoSuchAlgorithmException e) { |
| throw new RuntimeException(e); |
| } |
| |
| } |
| |
| /* (non-Javadoc) |
| * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext) |
| */ |
| @Override |
| public void close(TaskAttemptContext arg0) throws IOException, |
| InterruptedException { |
| out.close(); |
| } |
| |
| /* (non-Javadoc) |
| * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object) |
| */ |
| @Override |
| public void write(WritableComparable wc, Tuple t) throws IOException, |
| InterruptedException { |
| // we really only want to write the tuple (value) out here (and a sync syncMarker before that if necessary) |
| if (lastSyncPos == -1 || out.getPos() >= (lastSyncPos + syncMarkerInterval)) { |
| out.write(syncMarker); |
| lastSyncPos = out.getPos(); |
| } |
| sedes.writeDatum(out, t); |
| |
| } |
| |
| } |