@BeforeClass public void setUp() throws Exception { // Clean up ensureDirectoryExistsAndIsEmpty(_tmpDir); ensureDirectoryExistsAndIsEmpty(_segmentDir); ensureDirectoryExistsAndIsEmpty(_tarDir); // Start the cluster startCluster(); // Unpack the Avro files final List<File> avroFiles = unpackAvroData(_tmpDir, SEGMENT_COUNT); createTable(); // Load data into H2 ExecutorService executor = Executors.newCachedThreadPool(); setupH2AndInsertAvro(avroFiles, executor); // Create segments from Avro data buildSegmentsFromAvro(avroFiles, executor, 0, _segmentDir, _tarDir, "mytable", false, null); // Initialize query generator setupQueryGenerator(avroFiles, executor); executor.shutdown(); executor.awaitTermination(10, TimeUnit.MINUTES); // Set up a Helix spectator to count the number of segments that are uploaded and unlock the // latch once 12 segments are online final CountDownLatch latch = setupSegmentCountCountDownLatch("mytable", SEGMENT_COUNT); // Upload the segments int i = 0; for (String segmentName : _tarDir.list()) { System.out.println("Uploading segment " + (i++) + " : " + segmentName); File file = new File(_tarDir, segmentName); FileUploadUtils.sendSegmentFile( "localhost", "8998", segmentName, new FileInputStream(file), file.length()); } // Wait for all segments to be online latch.await(); TOTAL_DOCS = 115545; long timeInTwoMinutes = System.currentTimeMillis() + 2 * 60 * 1000L; long numDocs; while ((numDocs = getCurrentServingNumDocs()) < TOTAL_DOCS) { System.out.println("Current number of documents: " + numDocs); if (System.currentTimeMillis() < timeInTwoMinutes) { Thread.sleep(1000); } else { Assert.fail("Segments were not completely loaded within two minutes"); } } }
@Override public boolean execute() throws Exception { if (_controllerHost == null) { _controllerHost = NetUtil.getHostAddress(); } // Create a temp working directory. File tmpDir = File.createTempFile(SEGMENT_UPLOADER, null, FileUtils.getTempDirectory()); FileUtils.deleteQuietly(tmpDir); tmpDir.mkdir(); try { LOGGER.info("Executing command: " + toString()); File dir = new File(_segmentDir); File[] files = dir.listFiles(); for (File file : files) { File tgzFile = file; if (file.isDirectory()) { LOGGER.info("Compressing segment {}", file.getName()); String srcDir = file.getAbsolutePath(); String tgzFileName = TarGzCompressionUtils.createTarGzOfDirectory( srcDir, tmpDir.getAbsolutePath() + File.separator + file.getName() + TAR_GZIP); tgzFile = new File(tgzFileName); } LOGGER.info("Uploading segment {}", tgzFile.getName()); FileUploadUtils.sendSegmentFile( _controllerHost, _controllerPort, tgzFile.getName(), new FileInputStream(tgzFile), tgzFile.length()); } } catch (Exception e) { LOGGER.error("Exception caught while uploading segment {}", _segmentDir, e); } finally { // Delete the temporary working directory. FileUtils.deleteQuietly(tmpDir); } return true; }