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.replication.regionserver;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertTrue;
022import java.io.IOException;
023import java.net.SocketAddress;
024import java.util.ArrayList;
025import java.util.List;
026import java.util.concurrent.CompletableFuture;
027import java.util.concurrent.atomic.AtomicBoolean;
028import java.util.concurrent.atomic.AtomicInteger;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.CellBuilder;
032import org.apache.hadoop.hbase.CellBuilderFactory;
033import org.apache.hadoop.hbase.CellBuilderType;
034import org.apache.hadoop.hbase.CellScanner;
035import org.apache.hadoop.hbase.HBaseClassTestRule;
036import org.apache.hadoop.hbase.HBaseConfiguration;
037import org.apache.hadoop.hbase.Stoppable;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
040import org.apache.hadoop.hbase.client.AsyncClusterConnection;
041import org.apache.hadoop.hbase.client.AsyncConnection;
042import org.apache.hadoop.hbase.client.AsyncTable;
043import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
044import org.apache.hadoop.hbase.client.DummyAsyncClusterConnection;
045import org.apache.hadoop.hbase.client.DummyAsyncTable;
046import org.apache.hadoop.hbase.client.DummyConnectionRegistry;
047import org.apache.hadoop.hbase.client.Row;
048import org.apache.hadoop.hbase.security.User;
049import org.apache.hadoop.hbase.testclassification.ReplicationTests;
050import org.apache.hadoop.hbase.testclassification.SmallTests;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.junit.ClassRule;
053import org.junit.Rule;
054import org.junit.Test;
055import org.junit.experimental.categories.Category;
056import org.junit.rules.TestName;
057import org.slf4j.Logger;
058import org.slf4j.LoggerFactory;
059import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
060import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
061
062/**
063 * Simple test of sink-side wal entry filter facility.
064 */
065@Category({ ReplicationTests.class, SmallTests.class })
066public class TestWALEntrySinkFilter {
067
068  @ClassRule
069  public static final HBaseClassTestRule CLASS_RULE =
070    HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class);
071
072  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSink.class);
073  @Rule
074  public TestName name = new TestName();
075  static final int BOUNDARY = 5;
076  static final AtomicInteger UNFILTERED = new AtomicInteger();
077  static final AtomicInteger FILTERED = new AtomicInteger();
078
079  /**
080   * Implemetentation of Stoppable to pass into ReplicationSink.
081   */
082  private static Stoppable STOPPABLE = new Stoppable() {
083    private final AtomicBoolean stop = new AtomicBoolean(false);
084
085    @Override
086    public boolean isStopped() {
087      return this.stop.get();
088    }
089
090    @Override
091    public void stop(String why) {
092      LOG.info("STOPPING BECAUSE: " + why);
093      this.stop.set(true);
094    }
095  };
096
097  /**
098   * Test filter. Filter will filter out any write time that is <= 5 (BOUNDARY). We count how many
099   * items we filter out and we count how many cells make it through for distribution way down below
100   * in the Table#batch implementation. Puts in place a custom DevNullConnection so we can insert
101   * our counting Table.
102   * @throws IOException
103   */
104  @Test
105  public void testWALEntryFilter() throws IOException {
106    Configuration conf = HBaseConfiguration.create();
107    // Make it so our filter is instantiated on construction of ReplicationSink.
108    conf.setClass(DummyConnectionRegistry.REGISTRY_IMPL_CONF_KEY, DevNullConnectionRegistry.class,
109        DummyConnectionRegistry.class);
110    conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
111        IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
112    conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL,
113        DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
114    ReplicationSink sink = new ReplicationSink(conf);
115    // Create some dumb walentries.
116    List<AdminProtos.WALEntry> entries = new ArrayList<>();
117    AdminProtos.WALEntry.Builder entryBuilder = AdminProtos.WALEntry.newBuilder();
118    // Need a tablename.
119    ByteString tableName =
120      ByteString.copyFromUtf8(TableName.valueOf(this.name.getMethodName()).toString());
121    // Add WALEdit Cells to Cells List. The way edits arrive at the sink is with protos
122    // describing the edit with all Cells from all edits aggregated in a single CellScanner.
123    final List<Cell> cells = new ArrayList<>();
124    int count = BOUNDARY * 2;
125    for (int i = 0; i < count; i++) {
126      byte[] bytes = Bytes.toBytes(i);
127      // Create a wal entry. Everything is set to the current index as bytes or int/long.
128      entryBuilder.clear();
129      entryBuilder.setKey(entryBuilder.getKeyBuilder().setLogSequenceNumber(i)
130        .setEncodedRegionName(ByteString.copyFrom(bytes)).setWriteTime(i).setTableName(tableName)
131        .build());
132      // Lets have one Cell associated with each WALEdit.
133      entryBuilder.setAssociatedCellCount(1);
134      entries.add(entryBuilder.build());
135      // We need to add a Cell per WALEdit to the cells array.
136      CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.DEEP_COPY);
137      // Make cells whose row, family, cell, value, and ts are == 'i'.
138      Cell cell = cellBuilder.setRow(bytes).setFamily(bytes).setQualifier(bytes)
139        .setType(Cell.Type.Put).setTimestamp(i).setValue(bytes).build();
140      cells.add(cell);
141    }
142    // Now wrap our cells array in a CellScanner that we can pass in to replicateEntries. It has
143    // all Cells from all the WALEntries made above.
144    CellScanner cellScanner = new CellScanner() {
145      // Set to -1 because advance gets called before current.
146      int index = -1;
147
148      @Override
149      public Cell current() {
150        return cells.get(index);
151      }
152
153      @Override
154      public boolean advance() throws IOException {
155        index++;
156        return index < cells.size();
157      }
158    };
159    // Call our sink.
160    sink.replicateEntries(entries, cellScanner, null, null, null);
161    // Check what made it through and what was filtered.
162    assertTrue(FILTERED.get() > 0);
163    assertTrue(UNFILTERED.get() > 0);
164    assertEquals(count, FILTERED.get() + UNFILTERED.get());
165  }
166
167  /**
168   * Simple filter that will filter out any entry wholse writeTime is <= 5.
169   */
170  public static class IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl
171      implements WALEntrySinkFilter {
172    public IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl() {
173    }
174
175    @Override
176    public void init(AsyncConnection conn) {
177      // Do nothing.
178    }
179
180    @Override
181    public boolean filter(TableName table, long writeTime) {
182      boolean b = writeTime <= BOUNDARY;
183      if (b) {
184        FILTERED.incrementAndGet();
185      }
186      return b;
187    }
188  }
189
190  public static class DevNullConnectionRegistry extends DummyConnectionRegistry {
191
192    public DevNullConnectionRegistry(Configuration conf) {
193    }
194
195    @Override
196    public CompletableFuture<String> getClusterId() {
197      return CompletableFuture.completedFuture("test");
198    }
199  }
200
201  public static class DevNullAsyncClusterConnection extends DummyAsyncClusterConnection {
202
203    private final Configuration conf;
204
205    public DevNullAsyncClusterConnection(Configuration conf, Object registry, String clusterId,
206        SocketAddress localAddress, User user) {
207      this.conf = conf;
208    }
209
210    @Override
211    public AsyncTable<AdvancedScanResultConsumer> getTable(TableName tableName) {
212      return new DummyAsyncTable<AdvancedScanResultConsumer>() {
213
214        @Override
215        public <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
216          List<T> list = new ArrayList<>(actions.size());
217          for (Row action : actions) {
218            // Row is the index of the loop above where we make WALEntry and Cells.
219            int row = Bytes.toInt(action.getRow());
220            assertTrue("" + row, row > BOUNDARY);
221            UNFILTERED.incrementAndGet();
222            list.add(null);
223          }
224          return CompletableFuture.completedFuture(list);
225        }
226      };
227    }
228
229    @Override
230    public Configuration getConfiguration() {
231      return conf;
232    }
233  }
234}