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.mapreduce; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertTrue; 022import static org.junit.Assert.fail; 023import static org.mockito.Mockito.any; 024import static org.mockito.Mockito.doAnswer; 025import static org.mockito.Mockito.mock; 026import static org.mockito.Mockito.when; 027 028import java.io.ByteArrayOutputStream; 029import java.io.PrintStream; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.fs.Path; 032import org.apache.hadoop.hbase.HBaseClassTestRule; 033import org.apache.hadoop.hbase.HBaseTestingUtility; 034import org.apache.hadoop.hbase.client.Put; 035import org.apache.hadoop.hbase.client.Result; 036import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 037import org.apache.hadoop.hbase.mapreduce.IndexBuilder.Map; 038import org.apache.hadoop.hbase.mapreduce.SampleUploader.Uploader; 039import org.apache.hadoop.hbase.testclassification.LargeTests; 040import org.apache.hadoop.hbase.testclassification.MapReduceTests; 041import org.apache.hadoop.hbase.util.Bytes; 042import org.apache.hadoop.hbase.util.LauncherSecurityManager; 043import org.apache.hadoop.io.LongWritable; 044import org.apache.hadoop.io.Text; 045import org.apache.hadoop.mapreduce.Job; 046import org.apache.hadoop.mapreduce.Mapper; 047import org.apache.hadoop.mapreduce.Mapper.Context; 048import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; 049import org.junit.ClassRule; 050import org.junit.Test; 051import org.junit.experimental.categories.Category; 052import org.mockito.invocation.InvocationOnMock; 053import org.mockito.stubbing.Answer; 054 055@Category({ MapReduceTests.class, LargeTests.class }) 056public class TestMapReduceExamples { 057 @ClassRule 058 public static final HBaseClassTestRule CLASS_RULE = 059 HBaseClassTestRule.forClass(TestMapReduceExamples.class); 060 061 private static HBaseTestingUtility util = new HBaseTestingUtility(); 062 063 /** 064 * Test SampleUploader from examples 065 */ 066 @SuppressWarnings("unchecked") 067 @Test 068 public void testSampleUploader() throws Exception { 069 Configuration configuration = new Configuration(); 070 Uploader uploader = new Uploader(); 071 Mapper<LongWritable, Text, ImmutableBytesWritable, Put>.Context ctx = mock(Context.class); 072 doAnswer(new Answer<Void>() { 073 074 @Override 075 public Void answer(InvocationOnMock invocation) throws Throwable { 076 ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0); 077 Put put = (Put) invocation.getArgument(1); 078 assertEquals("row", Bytes.toString(writer.get())); 079 assertEquals("row", Bytes.toString(put.getRow())); 080 return null; 081 } 082 }).when(ctx).write(any(), any()); 083 084 uploader.map(null, new Text("row,family,qualifier,value"), ctx); 085 086 Path dir = util.getDataTestDirOnTestFS("testSampleUploader"); 087 088 String[] args = { dir.toString(), "simpleTable" }; 089 Job job = SampleUploader.configureJob(configuration, args); 090 assertEquals(SequenceFileInputFormat.class, job.getInputFormatClass()); 091 } 092 093 /** 094 * Test main method of SampleUploader. 095 */ 096 @Test 097 public void testMainSampleUploader() throws Exception { 098 PrintStream oldPrintStream = System.err; 099 SecurityManager SECURITY_MANAGER = System.getSecurityManager(); 100 LauncherSecurityManager newSecurityManager = new LauncherSecurityManager(); 101 System.setSecurityManager(newSecurityManager); 102 ByteArrayOutputStream data = new ByteArrayOutputStream(); 103 String[] args = {}; 104 System.setErr(new PrintStream(data)); 105 try { 106 System.setErr(new PrintStream(data)); 107 108 try { 109 SampleUploader.main(args); 110 fail("should be SecurityException"); 111 } catch (SecurityException e) { 112 assertEquals(-1, newSecurityManager.getExitCode()); 113 assertTrue(data.toString().contains("Wrong number of arguments:")); 114 assertTrue(data.toString().contains("Usage: SampleUploader <input> <tablename>")); 115 } 116 117 } finally { 118 System.setErr(oldPrintStream); 119 System.setSecurityManager(SECURITY_MANAGER); 120 } 121 122 } 123 124 /** 125 * Test IndexBuilder from examples 126 */ 127 @SuppressWarnings("unchecked") 128 @Test 129 public void testIndexBuilder() throws Exception { 130 Configuration configuration = new Configuration(); 131 String[] args = { "tableName", "columnFamily", "column1", "column2" }; 132 IndexBuilder.configureJob(configuration, args); 133 assertEquals("tableName", configuration.get("index.tablename")); 134 assertEquals("tableName", configuration.get(TableInputFormat.INPUT_TABLE)); 135 assertEquals("column1,column2", configuration.get("index.fields")); 136 137 Map map = new Map(); 138 ImmutableBytesWritable rowKey = new ImmutableBytesWritable(Bytes.toBytes("test")); 139 Mapper<ImmutableBytesWritable, Result, ImmutableBytesWritable, Put>.Context ctx = 140 mock(Context.class); 141 when(ctx.getConfiguration()).thenReturn(configuration); 142 doAnswer(new Answer<Void>() { 143 144 @Override 145 public Void answer(InvocationOnMock invocation) throws Throwable { 146 ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0); 147 Put put = (Put) invocation.getArgument(1); 148 assertEquals("tableName-column1", Bytes.toString(writer.get())); 149 assertEquals("test", Bytes.toString(put.getRow())); 150 return null; 151 } 152 }).when(ctx).write(any(), any()); 153 Result result = mock(Result.class); 154 when(result.getValue(Bytes.toBytes("columnFamily"), Bytes.toBytes("column1"))) 155 .thenReturn(Bytes.toBytes("test")); 156 map.setup(ctx); 157 map.map(rowKey, result, ctx); 158 } 159 160 /** 161 * Test main method of IndexBuilder 162 */ 163 @Test 164 public void testMainIndexBuilder() throws Exception { 165 PrintStream oldPrintStream = System.err; 166 SecurityManager SECURITY_MANAGER = System.getSecurityManager(); 167 LauncherSecurityManager newSecurityManager = new LauncherSecurityManager(); 168 System.setSecurityManager(newSecurityManager); 169 ByteArrayOutputStream data = new ByteArrayOutputStream(); 170 String[] args = {}; 171 System.setErr(new PrintStream(data)); 172 try { 173 System.setErr(new PrintStream(data)); 174 try { 175 IndexBuilder.main(args); 176 fail("should be SecurityException"); 177 } catch (SecurityException e) { 178 assertEquals(-1, newSecurityManager.getExitCode()); 179 assertTrue(data.toString().contains("arguments supplied, required: 3")); 180 assertTrue(data.toString() 181 .contains("Usage: IndexBuilder <TABLE_NAME> <COLUMN_FAMILY> <ATTR> [<ATTR> ...]")); 182 } 183 } finally { 184 System.setErr(oldPrintStream); 185 System.setSecurityManager(SECURITY_MANAGER); 186 } 187 } 188}