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