You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by se...@apache.org on 2013/04/30 20:34:59 UTC

svn commit: r1477762 [6/6] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-common/src/main/java/org/apache/hadoop/hbase/codec/ hbase-common/src/main/resources/ hbase-protocol/src/main/java/org/apache/hadoop...

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java Tue Apr 30 18:34:58 2013
@@ -491,7 +491,7 @@ public class ReplicationSource extends T
       HLogKey logKey = entry.getKey();
       // don't replicate if the log entries originated in the peer
       if (!logKey.getClusterId().equals(peerClusterId)) {
-        removeNonReplicableEdits(edit);
+        removeNonReplicableEdits(entry);
         // Don't replicate catalog entries, if the WALEdit wasn't
         // containing anything to replicate and if we're currently not set to replicate
         if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
@@ -666,12 +666,12 @@ public class ReplicationSource extends T
 
   /**
    * We only want KVs that are scoped other than local
-   * @param edit The KV to check for replication
+   * @param entry The entry to check for replication
    */
-  protected void removeNonReplicableEdits(WALEdit edit) {
-    NavigableMap<byte[], Integer> scopes = edit.getScopes();
-    List<KeyValue> kvs = edit.getKeyValues();
-    for (int i = edit.size()-1; i >= 0; i--) {
+  protected void removeNonReplicableEdits(HLog.Entry entry) {
+    NavigableMap<byte[], Integer> scopes = entry.getKey().getScopes();
+    List<KeyValue> kvs = entry.getEdit().getKeyValues();
+    for (int i = kvs.size()-1; i >= 0; i--) {
       KeyValue kv = kvs.get(i);
       // The scope will be null or empty if
       // there's nothing to replicate in that WALEdit

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java Tue Apr 30 18:34:58 2013
@@ -110,34 +110,25 @@ public class TestHLogRecordReader {
    */
   @Test
   public void testPartialRead() throws Exception {
-    HLog log = HLogFactory.createHLog(fs, hbaseDir,
-                                      logName, conf);
+    HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf);
     long ts = System.currentTimeMillis();
     WALEdit edit = new WALEdit();
-    edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
-        ts, value));
-    log.append(info, tableName, edit,
-      ts, htd);
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
+    log.append(info, tableName, edit, ts, htd);
     edit = new WALEdit();
-    edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
-        ts+1, value));
-    log.append(info, tableName, edit,
-        ts+1, htd);
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
+    log.append(info, tableName, edit, ts+1, htd);
     log.rollWriter();
 
     Thread.sleep(1);
     long ts1 = System.currentTimeMillis();
 
     edit = new WALEdit();
-    edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"),
-        ts1+1, value));
-    log.append(info, tableName, edit,
-        ts1+1, htd);
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
+    log.append(info, tableName, edit, ts1+1, htd);
     edit = new WALEdit();
-    edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"),
-        ts1+2, value));
-    log.append(info, tableName, edit,
-        ts1+2, htd);
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
+    log.append(info, tableName, edit, ts1+2, htd);
     log.close();
 
     HLogInputFormat input = new HLogInputFormat();
@@ -229,8 +220,11 @@ public class TestHLogRecordReader {
 
     for (byte[] column : columns) {
       assertTrue(reader.nextKeyValue());
-      assertTrue(Bytes
-          .equals(column, reader.getCurrentValue().getKeyValues().get(0).getQualifier()));
+      KeyValue kv = reader.getCurrentValue().getKeyValues().get(0);
+      if (!Bytes.equals(column, kv.getQualifier())) {
+        assertTrue("expected [" + Bytes.toString(column) + "], actual ["
+            + Bytes.toString(kv.getQualifier()) + "]", false);
+      }
     }
     assertFalse(reader.nextKeyValue());
     reader.close();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Tue Apr 30 18:34:58 2013
@@ -84,7 +84,7 @@ import org.apache.hadoop.hbase.monitorin
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.WAL.CompactionDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/FaultySequenceFileLogReader.java Tue Apr 30 18:34:58 2013
@@ -41,14 +41,12 @@ public class FaultySequenceFileLogReader
 
   @Override
   public HLog.Entry next(HLog.Entry reuse) throws IOException {
-    this.entryStart = this.reader.getPosition();
+    this.entryStart = this.getPosition();
     boolean b = true;
 
     if (nextQueue.isEmpty()) { // Read the whole thing at once and fake reading
       while (b == true) {
-        HLogKey key = HLogUtil.newKey(conf);
-        WALEdit val = new WALEdit();
-        HLog.Entry e = new HLog.Entry(key, val);
+        HLog.Entry e = new HLog.Entry(new HLogKey(), new WALEdit());
         if (compressionContext != null) {
           e.setCompressionContext(compressionContext);
         }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/InstrumentedSequenceFileLogWriter.java Tue Apr 30 18:34:58 2013
@@ -22,12 +22,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.util.Bytes;
 
-public class InstrumentedSequenceFileLogWriter extends SequenceFileLogWriter {
+public class InstrumentedSequenceFileLogWriter extends ProtobufLogWriter {
 
   public InstrumentedSequenceFileLogWriter() {
-    super(HLogKey.class);
+    super();
   }
-  
+
   public static boolean activateFailure = false;
   @Override
     public void append(HLog.Entry entry) throws IOException {

Added: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java?rev=1477762&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java (added)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java Tue Apr 30 18:34:58 2013
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Metadata;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Implementation of {@link HLog.Writer} that delegates to
+ * SequenceFile.Writer. Legacy implementation only used for compat tests.
+ */
+@InterfaceAudience.Private
+public class SequenceFileLogWriter implements HLog.Writer {
+  private final Log LOG = LogFactory.getLog(this.getClass());
+  // The sequence file we delegate to.
+  private SequenceFile.Writer writer;
+  // This is the FSDataOutputStream instance that is the 'out' instance
+  // in the SequenceFile.Writer 'writer' instance above.
+  private FSDataOutputStream writer_out;
+
+  // Legacy stuff from pre-PB WAL metadata.
+  private static final Text WAL_VERSION_KEY = new Text("version");
+  private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
+  private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
+  
+  /**
+   * Context used by our wal dictionary compressor.  Null if we're not to do
+   * our custom dictionary compression.  This custom WAL compression is distinct
+   * from sequencefile native compression.
+   */
+  private CompressionContext compressionContext;
+
+  /**
+   * Default constructor.
+   */
+  public SequenceFileLogWriter() {
+    super();
+  }
+
+  /**
+   * Create sequence file Metadata for our WAL file with version and compression
+   * type (if any).
+   * @param conf
+   * @param compress
+   * @return Metadata instance.
+   */
+  private static Metadata createMetadata(final Configuration conf,
+      final boolean compress) {
+    TreeMap<Text, Text> metaMap = new TreeMap<Text, Text>();
+    metaMap.put(WAL_VERSION_KEY, new Text("1"));
+    if (compress) {
+      // Currently we only do one compression type.
+      metaMap.put(WAL_COMPRESSION_TYPE_KEY, DICTIONARY_COMPRESSION_TYPE);
+    }
+    return new Metadata(metaMap);
+  }
+
+  @Override
+  public void init(FileSystem fs, Path path, Configuration conf)
+  throws IOException {
+    // Should we do our custom WAL compression?
+    boolean compress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    if (compress) {
+      try {
+        if (this.compressionContext == null) {
+          this.compressionContext = new CompressionContext(LRUDictionary.class);
+        } else {
+          this.compressionContext.clear();
+        }
+      } catch (Exception e) {
+        throw new IOException("Failed to initiate CompressionContext", e);
+      }
+    }
+
+    // Create a SF.Writer instance.
+    try {
+      // reflection for a version of SequenceFile.createWriter that doesn't
+      // automatically create the parent directory (see HBASE-2312)
+      this.writer = (SequenceFile.Writer) SequenceFile.class
+        .getMethod("createWriter", new Class[] {FileSystem.class,
+            Configuration.class, Path.class, Class.class, Class.class,
+            Integer.TYPE, Short.TYPE, Long.TYPE, Boolean.TYPE,
+            CompressionType.class, CompressionCodec.class, Metadata.class})
+        .invoke(null, new Object[] {fs, conf, path, HLogKey.class, WALEdit.class,
+            Integer.valueOf(fs.getConf().getInt("io.file.buffer.size", 4096)),
+            Short.valueOf((short)
+              conf.getInt("hbase.regionserver.hlog.replication",
+              fs.getDefaultReplication())),
+            Long.valueOf(conf.getLong("hbase.regionserver.hlog.blocksize",
+                fs.getDefaultBlockSize())),
+            Boolean.valueOf(false) /*createParent*/,
+            SequenceFile.CompressionType.NONE, new DefaultCodec(),
+            createMetadata(conf, compress)
+            });
+    } catch (InvocationTargetException ite) {
+      // function was properly called, but threw it's own exception
+      throw new IOException(ite.getCause());
+    } catch (Exception e) {
+      // ignore all other exceptions. related to reflection failure
+    }
+
+    // if reflection failed, use the old createWriter
+    if (this.writer == null) {
+      LOG.debug("new createWriter -- HADOOP-6840 -- not available");
+      this.writer = SequenceFile.createWriter(fs, conf, path,
+        HLogKey.class, WALEdit.class,
+        fs.getConf().getInt("io.file.buffer.size", 4096),
+        (short) conf.getInt("hbase.regionserver.hlog.replication",
+          fs.getDefaultReplication()),
+        conf.getLong("hbase.regionserver.hlog.blocksize",
+          fs.getDefaultBlockSize()),
+        SequenceFile.CompressionType.NONE,
+        new DefaultCodec(),
+        null,
+        createMetadata(conf, compress));
+    } else {
+      LOG.debug("using new createWriter -- HADOOP-6840");
+    }
+    
+    this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
+    LOG.debug("Path=" + path + ", compression=" + compress);
+  }
+
+  // Get at the private FSDataOutputStream inside in SequenceFile so we can
+  // call sync on it.  Make it accessible.
+  private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
+  throws IOException {
+    FSDataOutputStream out = null;
+    final Field fields [] = this.writer.getClass().getDeclaredFields();
+    final String fieldName = "out";
+    for (int i = 0; i < fields.length; ++i) {
+      if (fieldName.equals(fields[i].getName())) {
+        try {
+          // Make the 'out' field up in SF.Writer accessible.
+          fields[i].setAccessible(true);
+          out = (FSDataOutputStream)fields[i].get(this.writer);
+          break;
+        } catch (IllegalAccessException ex) {
+          throw new IOException("Accessing " + fieldName, ex);
+        } catch (SecurityException e) {
+          LOG.warn("Does not have access to out field from FSDataOutputStream",
+              e);
+        }
+      }
+    }
+    return out;
+  }
+
+  @Override
+  public void append(HLog.Entry entry) throws IOException {
+    entry.setCompressionContext(compressionContext);
+    try {
+      this.writer.append(entry.getKey(), entry.getEdit());
+    } catch (NullPointerException npe) {
+      // Concurrent close...
+      throw new IOException(npe);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.writer != null) {
+      try {
+        this.writer.close();
+      } catch (NullPointerException npe) {
+        // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
+        LOG.warn(npe);
+      }
+      this.writer = null;
+    }
+  }
+
+  @Override
+  public void sync() throws IOException {
+    try {
+      this.writer.syncFs();
+    } catch (NullPointerException npe) {
+      // Concurrent close...
+      throw new IOException(npe);
+    }
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    try {
+      return this.writer.getLength();
+    } catch (NullPointerException npe) {
+      // Concurrent close...
+      throw new IOException(npe);
+    }
+  }
+
+  /**
+   * @return The dfsclient out stream up inside SF.Writer made accessible, or
+   * null if not available.
+   */
+  public FSDataOutputStream getWriterFSDataOutputStream() {
+    return this.writer_out;
+  }
+}

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Tue Apr 30 18:34:58 2013
@@ -754,6 +754,73 @@ public class TestHLog  {
       log.append(hri, tableName, cols, timestamp, htd);
     }
   }
+  
+
+  /**
+   * @throws IOException
+   */
+  @Test
+  public void testReadLegacyLog() throws IOException {
+    final int columnCount = 5;
+    final int recordCount = 5;
+    final byte[] tableName = Bytes.toBytes("tablename");
+    final byte[] row = Bytes.toBytes("row");
+    long timestamp = System.currentTimeMillis();
+    Path path = new Path(dir, "temphlog");
+    SequenceFileLogWriter sflw = null;
+    HLog.Reader reader = null;
+    try {
+      HRegionInfo hri = new HRegionInfo(tableName,
+          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      fs.mkdirs(dir);
+      // Write log in pre-PB format.
+      sflw = new SequenceFileLogWriter();
+      sflw.init(fs, path, conf);
+      for (int i = 0; i < recordCount; ++i) {
+        HLogKey key = new HLogKey(
+            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
+        WALEdit edit = new WALEdit();
+        for (int j = 0; j < columnCount; ++j) {
+          if (i == 0) {
+            htd.addFamily(new HColumnDescriptor("column" + j));
+          }
+          String value = i + "" + j;
+          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
+        }
+        sflw.append(new HLog.Entry(key, edit));
+      }
+      sflw.sync();
+      sflw.close();
+
+      // Now read the log using standard means.
+      reader = HLogFactory.createReader(fs, path, conf);
+      assertTrue(reader instanceof SequenceFileLogReader);
+      for (int i = 0; i < recordCount; ++i) {
+        HLog.Entry entry = reader.next();
+        assertNotNull(entry);
+        assertEquals(columnCount, entry.getEdit().size());
+        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+        assertArrayEquals(tableName, entry.getKey().getTablename());
+        int idx = 0;
+        for (KeyValue val : entry.getEdit().getKeyValues()) {
+          assertTrue(Bytes.equals(row, val.getRow()));
+          String value = i + "" + idx;
+          assertArrayEquals(Bytes.toBytes(value), val.getValue());
+          idx++;
+        }
+      }
+      HLog.Entry entry = reader.next();
+      assertNull(entry);
+    } finally {
+      if (sflw != null) {
+        sflw.close();
+      }
+      if (reader != null) {
+        reader.close();
+      }
+    }
+  }
 
   static class DumbWALActionsListener implements WALActionsListener {
     int increments = 0;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Tue Apr 30 18:34:58 2013
@@ -652,7 +652,7 @@ public class TestHLogSplit {
     int actualCount = 0;
     HLog.Reader in = HLogFactory.createReader(fs, splitLog, conf);
     @SuppressWarnings("unused")
-	HLog.Entry entry;
+    HLog.Entry entry;
     while ((entry = in.next()) != null) ++actualCount;
     assertEquals(entryCount-1, actualCount);
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java Tue Apr 30 18:34:58 2013
@@ -100,8 +100,7 @@ public class TestWALActionsListener {
       HTableDescriptor htd = new HTableDescriptor();
       htd.addFamily(new HColumnDescriptor(b));
 
-      HLogKey key = new HLogKey(b,b, 0, 0, HConstants.DEFAULT_CLUSTER_ID);
-      hlog.append(hri, key, edit, htd, true);
+      hlog.append(hri, b, edit, 0, htd);
       if (i == 10) {
         hlog.registerWALActionsListener(laterobserver);
       }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Tue Apr 30 18:34:58 2013
@@ -827,7 +827,7 @@ public class TestWALReplay {
         "The sequence number of the recoverd.edits and the current edit seq should be same",
         lastestSeqNumber, editCount);
   }
-  
+
   static class MockHLog extends FSHLog {
     boolean doCompleteCacheFlush = false;
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java?rev=1477762&r1=1477761&r2=1477762&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java Tue Apr 30 18:34:58 2013
@@ -191,7 +191,7 @@ public class TestReplicationSourceManage
       LOG.info(i);
       HLogKey key = new HLogKey(hri.getRegionName(), test, seq++,
           System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
-      hlog.append(hri, key, edit, htd, true);
+      hlog.append(hri, test, edit, System.currentTimeMillis(), htd);
     }
 
     // Simulate a rapid insert that's followed
@@ -202,9 +202,7 @@ public class TestReplicationSourceManage
     LOG.info(baseline + " and " + time);
 
     for (int i = 0; i < 3; i++) {
-      HLogKey key = new HLogKey(hri.getRegionName(), test, seq++,
-          System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
-      hlog.append(hri, key, edit, htd, true);
+      hlog.append(hri, test, edit, System.currentTimeMillis(), htd);
     }
 
     assertEquals(6, manager.getHLogs().get(slaveId).size());
@@ -214,9 +212,7 @@ public class TestReplicationSourceManage
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
         "1", 0, false, false);
 
-    HLogKey key = new HLogKey(hri.getRegionName(), test, seq++,
-        System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
-    hlog.append(hri, key, edit, htd, true);
+    hlog.append(hri, test, edit, System.currentTimeMillis(), htd);
 
     assertEquals(1, manager.getHLogs().size());