hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mberto...@apache.org
Subject [1/2] hbase git commit: HBASE-13181 TestHRegionReplayEvents.testReplayBulkLoadEvent fails frequently
Date Tue, 10 Mar 2015 17:32:37 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1 78d6a3402 -> c4369d9a9
  refs/heads/master 29104da01 -> 7fa1a6b96


HBASE-13181 TestHRegionReplayEvents.testReplayBulkLoadEvent fails frequently


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7fa1a6b9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7fa1a6b9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7fa1a6b9

Branch: refs/heads/master
Commit: 7fa1a6b968bfde4f165d69a3840503ab0565349a
Parents: 29104da
Author: Matteo Bertozzi <matteo.bertozzi@cloudera.com>
Authored: Tue Mar 10 17:21:34 2015 +0000
Committer: Matteo Bertozzi <matteo.bertozzi@cloudera.com>
Committed: Tue Mar 10 17:21:34 2015 +0000

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/TestBulkLoad.java | 35 ++++++++++++--------
 .../regionserver/TestHRegionReplayEvents.java   | 23 ++++++++-----
 2 files changed, 36 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7fa1a6b9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 931306c..5629050 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -90,7 +90,7 @@ public class TestBulkLoad {
   private final Expectations callOnce;
   @Rule
   public TestName name = new TestName();
-  
+
   public TestBulkLoad() throws IOException {
     callOnce = new Expectations() {
       {
@@ -233,17 +233,24 @@ public class TestBulkLoad {
     HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
     // TODO We need a way to do this without creating files
     File hFileLocation = testFolder.newFile();
-    hFileFactory.withOutputStream(new FSDataOutputStream(new FileOutputStream(hFileLocation)));
-    hFileFactory.withFileContext(new HFileContext());
-    HFile.Writer writer = hFileFactory.create();
-
-    writer.append(new KeyValue(CellUtil.createCell(randomBytes,
-        family,
-        randomBytes,
-        0l,
-        KeyValue.Type.Put.getCode(),
-        randomBytes)));
-    writer.close();
+    FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation));
+    try {
+      hFileFactory.withOutputStream(out);
+      hFileFactory.withFileContext(new HFileContext());
+      HFile.Writer writer = hFileFactory.create();
+      try {
+        writer.append(new KeyValue(CellUtil.createCell(randomBytes,
+            family,
+            randomBytes,
+            0l,
+            KeyValue.Type.Put.getCode(),
+            randomBytes)));
+      } finally {
+        writer.close();
+      }
+    } finally {
+      out.close();
+    }
     return hFileLocation.getAbsoluteFile().getAbsolutePath();
   }
 
@@ -286,7 +293,7 @@ public class TestBulkLoad {
       assertNotNull(desc);
 
       if (tableName != null) {
-        assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(),

+        assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(),
           tableName));
       }
 
@@ -297,7 +304,7 @@ public class TestBulkLoad {
         assertTrue(Bytes.equals(Bytes.toBytes(store.getStoreHomeDir()), familyName));
         assertEquals(storeFileNames.size(), store.getStoreFileCount());
       }
-      
+
       return true;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fa1a6b9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index b877d5a..b89ba00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -1452,7 +1452,7 @@ public class TestHRegionReplayEvents {
     // close the region and open again.
     primaryRegion.close();
     primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss,
null);
-    
+
     // bulk load a file into primary region
     Random random = new Random();
     byte[] randomValues = new byte[20];
@@ -1515,13 +1515,20 @@ public class TestHRegionReplayEvents {
     HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(TEST_UTIL.getConfiguration());
     // TODO We need a way to do this without creating files
     Path testFile = new Path(testPath, UUID.randomUUID().toString());
-    hFileFactory.withOutputStream(TEST_UTIL.getTestFileSystem().create(testFile));
-    hFileFactory.withFileContext(new HFileContext());
-    HFile.Writer writer = hFileFactory.create();
-
-    writer.append(new KeyValue(CellUtil.createCell(valueBytes, family, valueBytes, 0l,
-      KeyValue.Type.Put.getCode(), valueBytes)));
-    writer.close();
+    FSDataOutputStream out = TEST_UTIL.getTestFileSystem().create(testFile);
+    try {
+      hFileFactory.withOutputStream(out);
+      hFileFactory.withFileContext(new HFileContext());
+      HFile.Writer writer = hFileFactory.create();
+      try {
+        writer.append(new KeyValue(CellUtil.createCell(valueBytes, family, valueBytes, 0l,
+          KeyValue.Type.Put.getCode(), valueBytes)));
+      } finally {
+        writer.close();
+      }
+    } finally {
+      out.close();
+    }
     return testFile.toString();
   }
 


Mime
View raw message