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.client;
019
020import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
021import static org.junit.Assert.assertArrayEquals;
022import static org.junit.Assert.assertEquals;
023import static org.junit.Assert.assertFalse;
024import static org.junit.Assert.assertNull;
025import static org.junit.Assert.assertThrows;
026import static org.junit.Assert.assertTrue;
027import static org.junit.Assert.fail;
028
029import java.io.IOException;
030import java.util.List;
031import java.util.regex.Pattern;
032import org.apache.hadoop.hbase.DoNotRetryIOException;
033import org.apache.hadoop.hbase.HBaseClassTestRule;
034import org.apache.hadoop.hbase.HConstants;
035import org.apache.hadoop.hbase.HRegionLocation;
036import org.apache.hadoop.hbase.InvalidFamilyOperationException;
037import org.apache.hadoop.hbase.MetaTableAccessor;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.TableNotDisabledException;
040import org.apache.hadoop.hbase.TableNotEnabledException;
041import org.apache.hadoop.hbase.TableNotFoundException;
042import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
043import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
044import org.apache.hadoop.hbase.testclassification.ClientTests;
045import org.apache.hadoop.hbase.testclassification.LargeTests;
046import org.apache.hadoop.hbase.util.Bytes;
047import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
048import org.junit.ClassRule;
049import org.junit.Test;
050import org.junit.experimental.categories.Category;
051import org.slf4j.Logger;
052import org.slf4j.LoggerFactory;
053
054@Category({ LargeTests.class, ClientTests.class })
055public class TestAdmin3 extends TestAdminBase {
056
057  @ClassRule
058  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestAdmin3.class);
059
060  private static final Logger LOG = LoggerFactory.getLogger(TestAdmin3.class);
061
062  @Test
063  public void testDisableAndEnableTable() throws IOException {
064    final byte[] row = Bytes.toBytes("row");
065    final byte[] qualifier = Bytes.toBytes("qualifier");
066    final byte[] value = Bytes.toBytes("value");
067    final TableName table = TableName.valueOf(name.getMethodName());
068    Table ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
069    Put put = new Put(row);
070    put.addColumn(HConstants.CATALOG_FAMILY, qualifier, value);
071    ht.put(put);
072    Get get = new Get(row);
073    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
074    ht.get(get);
075
076    ADMIN.disableTable(ht.getName());
077    assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster().getMaster()
078      .getTableStateManager().isTableState(ht.getName(), TableState.State.DISABLED));
079    assertEquals(TableState.State.DISABLED, getStateFromMeta(table));
080
081    // Test that table is disabled
082    get = new Get(row);
083    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
084    boolean ok = false;
085    try {
086      ht.get(get);
087    } catch (TableNotEnabledException e) {
088      ok = true;
089    }
090    ok = false;
091    // verify that scan encounters correct exception
092    Scan scan = new Scan();
093    try {
094      ResultScanner scanner = ht.getScanner(scan);
095      Result res = null;
096      do {
097        res = scanner.next();
098      } while (res != null);
099    } catch (TableNotEnabledException e) {
100      ok = true;
101    }
102    assertTrue(ok);
103    ADMIN.enableTable(table);
104    assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster().getMaster()
105      .getTableStateManager().isTableState(ht.getName(), TableState.State.ENABLED));
106    assertEquals(TableState.State.ENABLED, getStateFromMeta(table));
107
108    // Test that table is enabled
109    try {
110      ht.get(get);
111    } catch (RetriesExhaustedException e) {
112      ok = false;
113    }
114    assertTrue(ok);
115    ht.close();
116  }
117
118  @Test
119  public void testDisableAndEnableTables() throws IOException {
120    final byte[] row = Bytes.toBytes("row");
121    final byte[] qualifier = Bytes.toBytes("qualifier");
122    final byte[] value = Bytes.toBytes("value");
123    final TableName table1 = TableName.valueOf(name.getMethodName() + "1");
124    final TableName table2 = TableName.valueOf(name.getMethodName() + "2");
125    Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
126    Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
127    Put put = new Put(row);
128    put.addColumn(HConstants.CATALOG_FAMILY, qualifier, value);
129    ht1.put(put);
130    ht2.put(put);
131    Get get = new Get(row);
132    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
133    ht1.get(get);
134    ht2.get(get);
135
136    TableName[] tableNames = ADMIN.listTableNames(Pattern.compile("testDisableAndEnableTable.*"));
137    for (TableName tableName : tableNames) {
138      ADMIN.disableTable(tableName);
139    }
140
141    // Test that tables are disabled
142    get = new Get(row);
143    get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
144    boolean ok = false;
145    try {
146      ht1.get(get);
147      ht2.get(get);
148    } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
149      ok = true;
150    }
151
152    assertEquals(TableState.State.DISABLED, getStateFromMeta(table1));
153    assertEquals(TableState.State.DISABLED, getStateFromMeta(table2));
154
155    assertTrue(ok);
156    for (TableName tableName : tableNames) {
157      ADMIN.enableTable(tableName);
158    }
159
160    // Test that tables are enabled
161    try {
162      ht1.get(get);
163    } catch (IOException e) {
164      ok = false;
165    }
166    try {
167      ht2.get(get);
168    } catch (IOException e) {
169      ok = false;
170    }
171    assertTrue(ok);
172
173    ht1.close();
174    ht2.close();
175
176    assertEquals(TableState.State.ENABLED, getStateFromMeta(table1));
177    assertEquals(TableState.State.ENABLED, getStateFromMeta(table2));
178  }
179
180  /**
181   * Test retain assignment on enableTable.
182   */
183  @Test
184  public void testEnableTableRetainAssignment() throws IOException {
185    final TableName tableName = TableName.valueOf(name.getMethodName());
186    byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
187      new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
188      new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 } };
189    int expectedRegions = splitKeys.length + 1;
190    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
191      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
192    ADMIN.createTable(desc, splitKeys);
193
194    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
195      List<HRegionLocation> regions = l.getAllRegionLocations();
196
197      assertEquals(
198        "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
199        expectedRegions, regions.size());
200      // Disable table.
201      ADMIN.disableTable(tableName);
202      // Enable table, use retain assignment to assign regions.
203      ADMIN.enableTable(tableName);
204      List<HRegionLocation> regions2 = l.getAllRegionLocations();
205
206      // Check the assignment.
207      assertEquals(regions.size(), regions2.size());
208      assertTrue(regions2.containsAll(regions));
209    }
210  }
211
212  @Test
213  public void testEnableDisableAddColumnDeleteColumn() throws Exception {
214    final TableName tableName = TableName.valueOf(name.getMethodName());
215    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
216    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
217      Thread.sleep(10);
218    }
219    ADMIN.disableTable(tableName);
220    try {
221      TEST_UTIL.getConnection().getTable(tableName);
222    } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
223      // expected
224    }
225
226    ADMIN.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of("col2"));
227    ADMIN.enableTable(tableName);
228    try {
229      ADMIN.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
230    } catch (TableNotDisabledException e) {
231      LOG.info(e.toString(), e);
232    }
233    ADMIN.disableTable(tableName);
234    ADMIN.deleteTable(tableName);
235  }
236
237  @Test
238  public void testGetTableDescriptor() throws IOException {
239    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
240      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam1"))
241      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam2"))
242      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam3")).build();
243    ADMIN.createTable(htd);
244    Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
245    TableDescriptor confirmedHtd = table.getDescriptor();
246    // HBASE-26246 introduced persist of store file tracker into table descriptor
247    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL,
248      StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())).build();
249    assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
250    MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
251    table.close();
252  }
253
254  /**
255   * Verify schema change for read only table
256   */
257  @Test
258  public void testReadOnlyTableModify() throws IOException, InterruptedException {
259    final TableName tableName = TableName.valueOf(name.getMethodName());
260    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
261
262    // Make table read only
263    TableDescriptor htd =
264      TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName)).setReadOnly(true).build();
265    ADMIN.modifyTable(htd);
266
267    // try to modify the read only table now
268    htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
269      .setCompactionEnabled(false).build();
270    ADMIN.modifyTable(htd);
271    // Delete the table
272    ADMIN.disableTable(tableName);
273    ADMIN.deleteTable(tableName);
274    assertFalse(ADMIN.tableExists(tableName));
275  }
276
277  @Test
278  public void testDeleteLastColumnFamily() throws Exception {
279    final TableName tableName = TableName.valueOf(name.getMethodName());
280    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
281    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
282      Thread.sleep(10);
283    }
284
285    // test for enabled table
286    try {
287      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
288      fail("Should have failed to delete the only column family of a table");
289    } catch (InvalidFamilyOperationException ex) {
290      // expected
291    }
292
293    // test for disabled table
294    ADMIN.disableTable(tableName);
295
296    try {
297      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
298      fail("Should have failed to delete the only column family of a table");
299    } catch (InvalidFamilyOperationException ex) {
300      // expected
301    }
302
303    ADMIN.deleteTable(tableName);
304  }
305
306  @Test
307  public void testDeleteEditUnknownColumnFamilyAndOrTable() throws IOException {
308    // Test we get exception if we try to
309    final TableName nonexistentTable = TableName.valueOf("nonexistent");
310    final byte[] nonexistentColumn = Bytes.toBytes("nonexistent");
311    ColumnFamilyDescriptor nonexistentHcd = ColumnFamilyDescriptorBuilder.of(nonexistentColumn);
312    Exception exception = null;
313    try {
314      ADMIN.addColumnFamily(nonexistentTable, nonexistentHcd);
315    } catch (IOException e) {
316      exception = e;
317    }
318    assertTrue(exception instanceof TableNotFoundException);
319
320    exception = null;
321    try {
322      ADMIN.deleteTable(nonexistentTable);
323    } catch (IOException e) {
324      exception = e;
325    }
326    assertTrue(exception instanceof TableNotFoundException);
327
328    exception = null;
329    try {
330      ADMIN.deleteColumnFamily(nonexistentTable, nonexistentColumn);
331    } catch (IOException e) {
332      exception = e;
333    }
334    assertTrue(exception instanceof TableNotFoundException);
335
336    exception = null;
337    try {
338      ADMIN.disableTable(nonexistentTable);
339    } catch (IOException e) {
340      exception = e;
341    }
342    assertTrue(exception instanceof TableNotFoundException);
343
344    exception = null;
345    try {
346      ADMIN.enableTable(nonexistentTable);
347    } catch (IOException e) {
348      exception = e;
349    }
350    assertTrue(exception instanceof TableNotFoundException);
351
352    exception = null;
353    try {
354      ADMIN.modifyColumnFamily(nonexistentTable, nonexistentHcd);
355    } catch (IOException e) {
356      exception = e;
357    }
358    assertTrue(exception instanceof TableNotFoundException);
359
360    exception = null;
361    try {
362      TableDescriptor htd = TableDescriptorBuilder.newBuilder(nonexistentTable)
363        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
364      ADMIN.modifyTable(htd);
365    } catch (IOException e) {
366      exception = e;
367    }
368    assertTrue(exception instanceof TableNotFoundException);
369
370    // Now make it so at least the table exists and then do tests against a
371    // nonexistent column family -- see if we get right exceptions.
372    final TableName tableName =
373      TableName.valueOf(name.getMethodName() + EnvironmentEdgeManager.currentTime());
374    TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
375      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf")).build();
376    ADMIN.createTable(htd);
377    try {
378      exception = null;
379      try {
380        ADMIN.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
381      } catch (IOException e) {
382        exception = e;
383      }
384      assertTrue("found=" + exception.getClass().getName(),
385        exception instanceof InvalidFamilyOperationException);
386
387      exception = null;
388      try {
389        ADMIN.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
390      } catch (IOException e) {
391        exception = e;
392      }
393      assertTrue("found=" + exception.getClass().getName(),
394        exception instanceof InvalidFamilyOperationException);
395    } finally {
396      ADMIN.disableTable(tableName);
397      ADMIN.deleteTable(tableName);
398    }
399  }
400
401  private static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl";
402
403  private static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl";
404
405  private void verifyModifyTableResult(TableName tableName, byte[] family, byte[] qual, byte[] row,
406    byte[] value, String sft) throws IOException {
407    TableDescriptor td = ADMIN.getDescriptor(tableName);
408    assertEquals(sft, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
409    // no migration related configs
410    assertNull(td.getValue(SRC_IMPL));
411    assertNull(td.getValue(DST_IMPL));
412    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
413      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
414    }
415  }
416
417  @Test
418  public void testModifyTableStoreFileTracker() throws IOException {
419    TableName tableName = TableName.valueOf(name.getMethodName());
420    byte[] family = Bytes.toBytes("info");
421    byte[] qual = Bytes.toBytes("q");
422    byte[] row = Bytes.toBytes(0);
423    byte[] value = Bytes.toBytes(1);
424    try (Table table = TEST_UTIL.createTable(tableName, family)) {
425      table.put(new Put(row).addColumn(family, qual, value));
426    }
427    // change to FILE
428    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
429    verifyModifyTableResult(tableName, family, qual, row, value,
430      StoreFileTrackerFactory.Trackers.FILE.name());
431
432    // change to FILE again, should have no effect
433    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
434    verifyModifyTableResult(tableName, family, qual, row, value,
435      StoreFileTrackerFactory.Trackers.FILE.name());
436
437    // change to MIGRATION, and then to FILE
438    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
439      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
440        StoreFileTrackerFactory.Trackers.MIGRATION.name())
441      .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
442      .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build());
443    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
444    verifyModifyTableResult(tableName, family, qual, row, value,
445      StoreFileTrackerFactory.Trackers.FILE.name());
446
447    // change to MIGRATION, and then to DEFAULT
448    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
449      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
450        StoreFileTrackerFactory.Trackers.MIGRATION.name())
451      .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
452      .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build());
453    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.DEFAULT.name());
454    verifyModifyTableResult(tableName, family, qual, row, value,
455      StoreFileTrackerFactory.Trackers.DEFAULT.name());
456  }
457
458  private void verifyModifyColumnFamilyResult(TableName tableName, byte[] family, byte[] qual,
459    byte[] row, byte[] value, String sft) throws IOException {
460    TableDescriptor td = ADMIN.getDescriptor(tableName);
461    ColumnFamilyDescriptor cfd = td.getColumnFamily(family);
462    assertEquals(sft, cfd.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL));
463    // no migration related configs
464    assertNull(cfd.getConfigurationValue(SRC_IMPL));
465    assertNull(cfd.getConfigurationValue(DST_IMPL));
466    assertNull(cfd.getValue(SRC_IMPL));
467    assertNull(cfd.getValue(DST_IMPL));
468    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
469      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
470    }
471  }
472
473  @Test
474  public void testModifyColumnFamilyStoreFileTracker() throws IOException {
475    TableName tableName = TableName.valueOf(name.getMethodName());
476    byte[] family = Bytes.toBytes("info");
477    byte[] qual = Bytes.toBytes("q");
478    byte[] row = Bytes.toBytes(0);
479    byte[] value = Bytes.toBytes(1);
480    try (Table table = TEST_UTIL.createTable(tableName, family)) {
481      table.put(new Put(row).addColumn(family, qual, value));
482    }
483    // change to FILE
484    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
485      StoreFileTrackerFactory.Trackers.FILE.name());
486    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
487      StoreFileTrackerFactory.Trackers.FILE.name());
488
489    // change to FILE again, should have no effect
490    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
491      StoreFileTrackerFactory.Trackers.FILE.name());
492    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
493      StoreFileTrackerFactory.Trackers.FILE.name());
494
495    // change to MIGRATION, and then to FILE
496    TableDescriptor current = ADMIN.getDescriptor(tableName);
497    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current)
498      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
499        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
500          StoreFileTrackerFactory.Trackers.MIGRATION.name())
501        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
502        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
503      .build());
504    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
505      StoreFileTrackerFactory.Trackers.FILE.name());
506    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
507      StoreFileTrackerFactory.Trackers.FILE.name());
508
509    // change to MIGRATION, and then to DEFAULT
510    current = ADMIN.getDescriptor(tableName);
511    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current)
512      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
513        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
514          StoreFileTrackerFactory.Trackers.MIGRATION.name())
515        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
516        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
517      .build());
518    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
519      StoreFileTrackerFactory.Trackers.DEFAULT.name());
520    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
521      StoreFileTrackerFactory.Trackers.DEFAULT.name());
522  }
523
524  @Test
525  public void testModifyStoreFileTrackerError() throws IOException {
526    TableName tableName = TableName.valueOf(name.getMethodName());
527    byte[] family = Bytes.toBytes("info");
528    TEST_UTIL.createTable(tableName, family).close();
529
530    // table not exists
531    assertThrows(TableNotFoundException.class,
532      () -> ADMIN.modifyTableStoreFileTracker(TableName.valueOf("whatever"),
533        StoreFileTrackerFactory.Trackers.FILE.name()));
534    // family not exists
535    assertThrows(NoSuchColumnFamilyException.class,
536      () -> ADMIN.modifyColumnFamilyStoreFileTracker(tableName, Bytes.toBytes("not_exists"),
537        StoreFileTrackerFactory.Trackers.FILE.name()));
538    // to migration
539    assertThrows(DoNotRetryIOException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName,
540      StoreFileTrackerFactory.Trackers.MIGRATION.name()));
541    // disabled
542    ADMIN.disableTable(tableName);
543    assertThrows(TableNotEnabledException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName,
544      StoreFileTrackerFactory.Trackers.FILE.name()));
545  }
546}