hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e...@apache.org
Subject svn commit: r1565003 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
Date Wed, 05 Feb 2014 23:56:05 GMT
Author: enis
Date: Wed Feb  5 23:56:04 2014
New Revision: 1565003

URL: http://svn.apache.org/r1565003
Log:
HBASE-10471 Remove HTD.isAsyncLogFlush() from trunk

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1565003&r1=1565002&r2=1565003&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed
Feb  5 23:56:04 2014
@@ -597,34 +597,6 @@ public class HTableDescriptor implements
   }
 
   /**
-   * Check if async log edits are enabled on the table.
-   *
-   * @return true if that async log flush is enabled on the table
-   * @deprecated Since 0.96 we no longer have an explicity deferred log flush/sync functionality.
-   * Use {@link #getDurability()}.
-   */
-  public synchronized boolean isAsyncLogFlush() {
-    return getDurability() == Durability.ASYNC_WAL;
-  }
-
-  /**
-   * This is used to allowing the log edits syncing to the file system. Everytime
-   * an edit is sent to the server it is first sync'd to the file system by the
-   * log writer. This sync is an expensive operation and thus can be deferred so
-   * that the edits are kept in memory until the background async writer-sync-notifier
-   * threads do the sync and not explicitly flushed for every edit.
-   * <p>
-   * NOTE:- This option might result in data loss if the region server crashes
-   * before these pending edits in memory are flushed onto the filesystem.
-   * </p>
-   *
-   * @param isAsyncLogFlush
-   */
-  public synchronized void setAsyncLogFlush(final boolean isAsyncLogFlush) {
-    this.setDurability(isAsyncLogFlush ? Durability.ASYNC_WAL : DEFAULT_DURABLITY);
-  }
-
-  /**
    * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
    * @param durability enum value
    */

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java?rev=1565003&r1=1565002&r2=1565003&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
(original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
Wed Feb  5 23:56:04 2014
@@ -79,8 +79,8 @@ public class TestDurability {
     HLog wal = HLogFactory.createHLog(FS, DIR, "hlogdir",
         "hlogdir_archive", CONF);
     byte[] tableName = Bytes.toBytes("TestDurability");
-    HRegion region = createHRegion(tableName, "region", wal, false);
-    HRegion deferredRegion = createHRegion(tableName, "deferredRegion", wal, true);
+    HRegion region = createHRegion(tableName, "region", wal, Durability.USE_DEFAULT);
+    HRegion deferredRegion = createHRegion(tableName, "deferredRegion", wal, Durability.ASYNC_WAL);
 
     region.put(newPut(null));
     verifyHLogCount(wal, 1);
@@ -142,7 +142,7 @@ public class TestDurability {
     HLog wal = HLogFactory.createHLog(FS, DIR, "myhlogdir",
         "myhlogdir_archive", CONF);
     byte[] tableName = Bytes.toBytes("TestIncrement");
-    HRegion region = createHRegion(tableName, "increment", wal, false);
+    HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT);
 
     // col1: amount = 1, 1 write back to WAL
     Increment inc1 = new Increment(row1);
@@ -208,10 +208,10 @@ public class TestDurability {
 
   // lifted from TestAtomicOperation
   private HRegion createHRegion (byte [] tableName, String callingMethod,
-      HLog log, boolean isAsyncLogFlush)
+      HLog log, Durability durability)
     throws IOException {
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-      htd.setAsyncLogFlush(isAsyncLogFlush);
+      htd.setDurability(durability);
       HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
       htd.addFamily(hcd);
       HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);



Mime
View raw message