001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.wal; 019 020import java.io.EOFException; 021import java.io.IOException; 022import org.apache.hadoop.conf.Configuration; 023import org.apache.hadoop.fs.FileSystem; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.hbase.wal.WAL.Entry; 026import org.slf4j.Logger; 027import org.slf4j.LoggerFactory; 028 029/** 030 * A helper class for writing UTs, where we will eat the EOF and return null when reaching EOF, so 031 * in UTs we do not need to deal with partial WAL files if this does not affect the correctness. In 032 * production code you usually you should not do this, as it may cause data loss if you always 033 * ignore the EOFException. 034 */ 035public final class NoEOFWALStreamReader implements WALStreamReader { 036 037 private static final Logger LOG = LoggerFactory.getLogger(NoEOFWALStreamReader.class); 038 039 private WALStreamReader reader; 040 041 private NoEOFWALStreamReader(WALStreamReader reader) { 042 this.reader = reader; 043 } 044 045 @Override 046 public Entry next(Entry reuse) throws IOException { 047 try { 048 return reader.next(reuse); 049 } catch (EOFException e) { 050 LOG.warn("Got EOF while reading", e); 051 return null; 052 } 053 } 054 055 @Override 056 public long getPosition() throws IOException { 057 return reader.getPosition(); 058 } 059 060 @Override 061 public void close() { 062 reader.close(); 063 } 064 065 private int count() throws IOException { 066 int count = 0; 067 Entry entry = new Entry(); 068 while (next(entry) != null) { 069 count++; 070 } 071 return count; 072 } 073 074 public static NoEOFWALStreamReader create(FileSystem fs, Path path, Configuration conf) 075 throws IOException { 076 return new NoEOFWALStreamReader(WALFactory.createStreamReader(fs, path, conf)); 077 } 078 079 public static NoEOFWALStreamReader create(WALFactory walFactory, FileSystem fs, Path path) 080 throws IOException { 081 return new NoEOFWALStreamReader(walFactory.createStreamReader(fs, path)); 082 } 083 084 public static int count(FileSystem fs, Path path, Configuration conf) throws IOException { 085 try (NoEOFWALStreamReader reader = create(fs, path, conf)) { 086 return reader.count(); 087 } 088 } 089 090 public static int count(WALFactory walFactory, FileSystem fs, Path path) throws IOException { 091 try (NoEOFWALStreamReader reader = create(walFactory, fs, path)) { 092 return reader.count(); 093 } 094 } 095}