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.snapshot; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertTrue; 022 023import java.util.ArrayList; 024import java.util.Collection; 025import java.util.HashSet; 026import java.util.List; 027import java.util.Set; 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.HBaseClassTestRule; 030import org.apache.hadoop.hbase.testclassification.RegionServerTests; 031import org.apache.hadoop.hbase.testclassification.SmallTests; 032import org.apache.hadoop.hbase.util.Pair; 033import org.junit.ClassRule; 034import org.junit.Test; 035import org.junit.experimental.categories.Category; 036 037import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo; 038 039/** 040 * Test Export Snapshot Tool helpers 041 */ 042@Category({ RegionServerTests.class, SmallTests.class }) 043public class TestExportSnapshotHelpers { 044 045 @ClassRule 046 public static final HBaseClassTestRule CLASS_RULE = 047 HBaseClassTestRule.forClass(TestExportSnapshotHelpers.class); 048 049 /** 050 * Verfy the result of getBalanceSplits() method. The result are groups of files, used as input 051 * list for the "export" mappers. All the groups should have similar amount of data. The input 052 * list is a pair of file path and length. The getBalanceSplits() function sort it by length, and 053 * assign to each group a file, going back and forth through the groups. 054 */ 055 @Test 056 public void testBalanceSplit() { 057 // Create a list of files 058 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(21); 059 for (long i = 0; i <= 20; i++) { 060 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 061 .setHfile("file-" + i).build(); 062 files.add(new Pair<>(fileInfo, i)); 063 } 064 065 // Create 5 groups (total size 210) 066 // group 0: 20, 11, 10, 1 (total size: 42) 067 // group 1: 19, 12, 9, 2 (total size: 42) 068 // group 2: 18, 13, 8, 3 (total size: 42) 069 // group 3: 17, 12, 7, 4 (total size: 42) 070 // group 4: 16, 11, 6, 5 (total size: 42) 071 List<List<Pair<SnapshotFileInfo, Long>>> splits = ExportSnapshot.getBalancedSplits(files, 5); 072 assertEquals(5, splits.size()); 073 074 String[] split0 = new String[] { "file-20", "file-11", "file-10", "file-1", "file-0" }; 075 verifyBalanceSplit(splits.get(0), split0, 42); 076 String[] split1 = new String[] { "file-19", "file-12", "file-9", "file-2" }; 077 verifyBalanceSplit(splits.get(1), split1, 42); 078 String[] split2 = new String[] { "file-18", "file-13", "file-8", "file-3" }; 079 verifyBalanceSplit(splits.get(2), split2, 42); 080 String[] split3 = new String[] { "file-17", "file-14", "file-7", "file-4" }; 081 verifyBalanceSplit(splits.get(3), split3, 42); 082 String[] split4 = new String[] { "file-16", "file-15", "file-6", "file-5" }; 083 verifyBalanceSplit(splits.get(4), split4, 42); 084 } 085 086 private void verifyBalanceSplit(final List<Pair<SnapshotFileInfo, Long>> split, 087 final String[] expected, final long expectedSize) { 088 assertEquals(expected.length, split.size()); 089 long totalSize = 0; 090 for (int i = 0; i < expected.length; ++i) { 091 Pair<SnapshotFileInfo, Long> fileInfo = split.get(i); 092 assertEquals(expected[i], fileInfo.getFirst().getHfile()); 093 totalSize += fileInfo.getSecond(); 094 } 095 assertEquals(expectedSize, totalSize); 096 } 097 098 @Test 099 public void testGroupFilesForSplitsWithoutCustomFileGrouper() { 100 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 101 for (long i = 0; i < 10; i++) { 102 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 103 .setHfile("file-" + i).build(); 104 files.add(new Pair<>(fileInfo, i * 10)); 105 } 106 107 Configuration conf = new Configuration(); 108 conf.setInt("snapshot.export.format.splits", 3); 109 110 ExportSnapshot.ExportSnapshotInputFormat inputFormat = 111 new ExportSnapshot.ExportSnapshotInputFormat(); 112 Collection<List<Pair<SnapshotFileInfo, Long>>> groups = 113 inputFormat.groupFilesForSplits(conf, files); 114 115 assertEquals("Should create 3 groups", 3, groups.size()); 116 117 long totalSize = 0; 118 int totalFiles = 0; 119 for (List<Pair<SnapshotFileInfo, Long>> group : groups) { 120 for (Pair<SnapshotFileInfo, Long> file : group) { 121 totalSize += file.getSecond(); 122 totalFiles++; 123 } 124 } 125 126 assertEquals("All files should be included", 10, totalFiles); 127 assertEquals("Total size should be preserved", 450, totalSize); 128 } 129 130 @Test 131 public void testGroupFilesForSplitsWithCustomFileGrouper() { 132 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 133 for (long i = 0; i < 8; i++) { 134 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 135 .setHfile("file-" + i).build(); 136 files.add(new Pair<>(fileInfo, i * 5)); 137 } 138 139 Configuration conf = new Configuration(); 140 conf.setInt("snapshot.export.format.splits", 4); 141 conf.setClass("snapshot.export.input.file.grouper.class", TestCustomFileGrouper.class, 142 ExportSnapshot.CustomFileGrouper.class); 143 144 ExportSnapshot.ExportSnapshotInputFormat inputFormat = 145 new ExportSnapshot.ExportSnapshotInputFormat(); 146 Collection<List<Pair<SnapshotFileInfo, Long>>> groups = 147 inputFormat.groupFilesForSplits(conf, files); 148 149 assertEquals("Should create splits based on custom grouper output", 4, groups.size()); 150 151 long totalSize = 0; 152 int totalFiles = 0; 153 for (List<Pair<SnapshotFileInfo, Long>> group : groups) { 154 for (Pair<SnapshotFileInfo, Long> file : group) { 155 totalSize += file.getSecond(); 156 totalFiles++; 157 } 158 } 159 160 assertEquals("All files should be included", 8, totalFiles); 161 assertEquals("Total size should be preserved", 140, totalSize); 162 } 163 164 @Test 165 public void testFileLocationResolverWithNoopResolver() { 166 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 167 for (long i = 0; i < 3; i++) { 168 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 169 .setHfile("file-" + i).build(); 170 files.add(new Pair<>(fileInfo, i * 10)); 171 } 172 173 ExportSnapshot.NoopFileLocationResolver resolver = 174 new ExportSnapshot.NoopFileLocationResolver(); 175 Set<String> locations = resolver.getLocationsForInputFiles(files); 176 177 assertTrue("NoopFileLocationResolver should return empty locations", locations.isEmpty()); 178 } 179 180 @Test 181 public void testFileLocationResolverWithCustomResolver() { 182 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 183 for (long i = 0; i < 3; i++) { 184 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 185 .setHfile("file-" + i).build(); 186 files.add(new Pair<>(fileInfo, i * 10)); 187 } 188 189 TestFileLocationResolver resolver = new TestFileLocationResolver(); 190 Set<String> locations = resolver.getLocationsForInputFiles(files); 191 192 assertEquals("Should return expected locations", 2, locations.size()); 193 assertTrue("Should contain rack1", locations.contains("rack1")); 194 assertTrue("Should contain rack2", locations.contains("rack2")); 195 } 196 197 @Test 198 public void testInputSplitWithFileLocationResolver() { 199 List<Pair<SnapshotFileInfo, Long>> files = new ArrayList<>(); 200 for (long i = 0; i < 3; i++) { 201 SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE) 202 .setHfile("file-" + i).build(); 203 files.add(new Pair<>(fileInfo, i * 10)); 204 } 205 206 TestFileLocationResolver resolver = new TestFileLocationResolver(); 207 ExportSnapshot.ExportSnapshotInputFormat.ExportSnapshotInputSplit split = 208 new ExportSnapshot.ExportSnapshotInputFormat.ExportSnapshotInputSplit(files, resolver); 209 210 try { 211 String[] locations = split.getLocations(); 212 assertEquals("Should return 2 locations", 2, locations.length); 213 214 boolean hasRack1 = false; 215 boolean hasRack2 = false; 216 for (String location : locations) { 217 if ("rack1".equals(location)) { 218 hasRack1 = true; 219 } 220 if ("rack2".equals(location)) { 221 hasRack2 = true; 222 } 223 } 224 225 assertTrue("Should contain rack1", hasRack1); 226 assertTrue("Should contain rack2", hasRack2); 227 } catch (Exception e) { 228 throw new RuntimeException("Failed to get locations", e); 229 } 230 } 231 232 public static class TestCustomFileGrouper implements ExportSnapshot.CustomFileGrouper { 233 @Override 234 public Collection<Collection<Pair<SnapshotFileInfo, Long>>> 235 getGroupedInputFiles(Collection<Pair<SnapshotFileInfo, Long>> snapshotFiles) { 236 List<Collection<Pair<SnapshotFileInfo, Long>>> groups = new ArrayList<>(); 237 List<Pair<SnapshotFileInfo, Long>> group1 = new ArrayList<>(); 238 List<Pair<SnapshotFileInfo, Long>> group2 = new ArrayList<>(); 239 240 int count = 0; 241 for (Pair<SnapshotFileInfo, Long> file : snapshotFiles) { 242 if (count % 2 == 0) { 243 group1.add(file); 244 } else { 245 group2.add(file); 246 } 247 count++; 248 } 249 250 groups.add(group1); 251 groups.add(group2); 252 return groups; 253 } 254 } 255 256 public static class TestFileLocationResolver implements ExportSnapshot.FileLocationResolver { 257 @Override 258 public Set<String> getLocationsForInputFiles(Collection<Pair<SnapshotFileInfo, Long>> files) { 259 Set<String> locations = new HashSet<>(); 260 locations.add("rack1"); 261 locations.add("rack2"); 262 return locations; 263 } 264 } 265}