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.io; 019 020import static org.junit.jupiter.api.Assertions.assertFalse; 021import static org.junit.jupiter.api.Assertions.assertTrue; 022 023import java.io.IOException; 024import java.io.InputStream; 025import java.nio.ByteBuffer; 026import java.util.EnumSet; 027import org.apache.hadoop.fs.ByteBufferReadable; 028import org.apache.hadoop.fs.CanSetDropBehind; 029import org.apache.hadoop.fs.CanSetReadahead; 030import org.apache.hadoop.fs.CanUnbuffer; 031import org.apache.hadoop.fs.FSDataInputStream; 032import org.apache.hadoop.fs.FSInputStream; 033import org.apache.hadoop.fs.HasEnhancedByteBufferAccess; 034import org.apache.hadoop.fs.ReadOption; 035import org.apache.hadoop.fs.StreamCapabilities; 036import org.apache.hadoop.hbase.testclassification.MiscTests; 037import org.apache.hadoop.hbase.testclassification.SmallTests; 038import org.apache.hadoop.io.ByteBufferPool; 039import org.junit.Test; 040import org.junit.jupiter.api.Tag; 041 042@Tag(SmallTests.TAG) 043@Tag(MiscTests.TAG) 044public class TestFSDataInputStreamWrapper { 045 046 @Test 047 public void testUnbuffer() throws Exception { 048 InputStream pc = new ParentClass(); 049 InputStream noChecksumPc = new ParentClass(); 050 FSDataInputStreamWrapper fsdisw1 = 051 new FSDataInputStreamWrapper(new FSDataInputStream(pc), new FSDataInputStream(noChecksumPc)); 052 fsdisw1.unbuffer(); 053 // should have called main stream unbuffer, but not no-checksum 054 assertTrue(((ParentClass) pc).getIsCallUnbuffer()); 055 assertFalse(((ParentClass) noChecksumPc).getIsCallUnbuffer()); 056 // switch to checksums and call unbuffer again. should unbuffer the nochecksum stream now 057 fsdisw1.setShouldUseHBaseChecksum(); 058 fsdisw1.unbuffer(); 059 assertTrue(((ParentClass) noChecksumPc).getIsCallUnbuffer()); 060 fsdisw1.close(); 061 } 062 063 private class ParentClass extends FSInputStream implements ByteBufferReadable, CanSetDropBehind, 064 CanSetReadahead, HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities { 065 066 public boolean isCallUnbuffer = false; 067 068 public boolean getIsCallUnbuffer() { 069 return isCallUnbuffer; 070 } 071 072 @Override 073 public void unbuffer() { 074 isCallUnbuffer = true; 075 } 076 077 @Override 078 public int read() throws IOException { 079 return 0; 080 } 081 082 @Override 083 public ByteBuffer read(ByteBufferPool paramByteBufferPool, int paramInt, 084 EnumSet<ReadOption> paramEnumSet) throws IOException, UnsupportedOperationException { 085 return null; 086 } 087 088 @Override 089 public void releaseBuffer(ByteBuffer paramByteBuffer) { 090 091 } 092 093 @Override 094 public void setReadahead(Long paramLong) throws IOException, UnsupportedOperationException { 095 096 } 097 098 @Override 099 public void setDropBehind(Boolean paramBoolean) 100 throws IOException, UnsupportedOperationException { 101 102 } 103 104 @Override 105 public int read(ByteBuffer paramByteBuffer) throws IOException { 106 return 0; 107 } 108 109 @Override 110 public void seek(long paramLong) throws IOException { 111 112 } 113 114 @Override 115 public long getPos() throws IOException { 116 return 0; 117 } 118 119 @Override 120 public boolean seekToNewSource(long paramLong) throws IOException { 121 return false; 122 } 123 124 @Override 125 public boolean hasCapability(String s) { 126 return s.equals(StreamCapabilities.UNBUFFER); 127 } 128 } 129}