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