github.com/tsuna/gohbase@v0.0.0-20250731002811-4ffcadfba63e/integration_test.go (about)

     1  // Copyright (C) 2015  The GoHBase Authors.  All rights reserved.
     2  // This file is part of GoHBase.
     3  // Use of this source code is governed by the Apache License 2.0
     4  // that can be found in the COPYING file.
     5  
     6  //go:build integration
     7  
     8  package gohbase_test
     9  
    10  import (
    11  	"bytes"
    12  	"context"
    13  	"crypto/rand"
    14  	"encoding/json"
    15  	"flag"
    16  	"fmt"
    17  	"io"
    18  	"log/slog"
    19  	"math"
    20  	"os"
    21  	"os/exec"
    22  	"reflect"
    23  	"strconv"
    24  	"strings"
    25  	"sync"
    26  	"testing"
    27  	"time"
    28  
    29  	"github.com/stretchr/testify/assert"
    30  	"github.com/tsuna/gohbase"
    31  	"github.com/tsuna/gohbase/filter"
    32  	"github.com/tsuna/gohbase/hrpc"
    33  	"github.com/tsuna/gohbase/pb"
    34  	"github.com/tsuna/gohbase/region"
    35  	"google.golang.org/protobuf/proto"
    36  )
    37  
    38  var (
    39  	host      = flag.String("host", "localhost", "The location where HBase is running")
    40  	keySplits = [][]byte{[]byte("REVTEST-100"), []byte("REVTEST-200"), []byte("REVTEST-300")}
    41  	table     string
    42  )
    43  
    44  func init() {
    45  	table = fmt.Sprintf("gohbase_test_%d", time.Now().UnixNano())
    46  }
    47  
    48  const scannerLease = 5 * time.Second
    49  
    50  // CreateTable creates the given table with the given families
    51  func CreateTable(client gohbase.AdminClient, table string, cFamilies []string) error {
    52  	// If the table exists, delete it
    53  	DeleteTable(client, table)
    54  	// Don't check the error, since one will be returned if the table doesn't
    55  	// exist
    56  
    57  	cf := make(map[string]map[string]string, len(cFamilies))
    58  	for _, f := range cFamilies {
    59  		cf[f] = nil
    60  	}
    61  
    62  	// pre-split table for reverse scan test of region changes
    63  	ct := hrpc.NewCreateTable(context.Background(), []byte(table), cf, hrpc.SplitKeys(keySplits))
    64  	if err := client.CreateTable(ct); err != nil {
    65  		return err
    66  	}
    67  
    68  	return nil
    69  }
    70  
    71  // DeleteTable finds the HBase shell via the HBASE_HOME environment variable,
    72  // and disables and drops the given table
    73  func DeleteTable(client gohbase.AdminClient, table string) error {
    74  	dit := hrpc.NewDisableTable(context.Background(), []byte(table))
    75  	err := client.DisableTable(dit)
    76  	if err != nil {
    77  		if !strings.Contains(err.Error(), "TableNotEnabledException") {
    78  			return err
    79  		}
    80  	}
    81  
    82  	det := hrpc.NewDeleteTable(context.Background(), []byte(table))
    83  	err = client.DeleteTable(det)
    84  	if err != nil {
    85  		return err
    86  	}
    87  	return nil
    88  }
    89  
    90  func LocalRegionServersCmd(t *testing.T, action string, servers []string) {
    91  	hh := os.Getenv("HBASE_HOME")
    92  	args := append([]string{action}, servers...)
    93  	if err := exec.Command(hh+"/bin/local-regionservers.sh", args...).Run(); err != nil {
    94  		t.Errorf("failed to %s RS=%v: %v", action, servers, err)
    95  	}
    96  }
    97  
    98  func TestMain(m *testing.M) {
    99  	flag.Parse()
   100  
   101  	if host == nil {
   102  		panic("Host is not set!")
   103  	}
   104  
   105  	slog.SetLogLoggerLevel(slog.LevelDebug)
   106  
   107  	ac := gohbase.NewAdminClient(*host)
   108  
   109  	var err error
   110  	for {
   111  		err = CreateTable(ac, table, []string{"cf", "cf1", "cf2"})
   112  		if err != nil &&
   113  			(strings.Contains(err.Error(), "org.apache.hadoop.hbase.PleaseHoldException") ||
   114  				strings.Contains(err.Error(),
   115  					"org.apache.hadoop.hbase.ipc.ServerNotRunningYetException")) {
   116  			time.Sleep(time.Second)
   117  			continue
   118  		} else if err != nil {
   119  			panic(err)
   120  		} else {
   121  			break
   122  		}
   123  	}
   124  	res := m.Run()
   125  	err = DeleteTable(ac, table)
   126  	if err != nil {
   127  		panic(err)
   128  	}
   129  
   130  	os.Exit(res)
   131  }
   132  
   133  // Test retrieval of cluster status
   134  func TestClusterStatus(t *testing.T) {
   135  	ac := gohbase.NewAdminClient(*host)
   136  
   137  	stats, err := ac.ClusterStatus()
   138  	if err != nil {
   139  		t.Fatal(err)
   140  	}
   141  
   142  	//Sanity check the data coming back
   143  	if len(stats.GetMaster().GetHostName()) == 0 {
   144  		t.Fatal("Master hostname is empty in ClusterStatus")
   145  	}
   146  }
   147  
   148  func TestGet(t *testing.T) {
   149  	key := "row1"
   150  	val := []byte("1")
   151  	headers := map[string][]string{"cf": nil}
   152  	if host == nil {
   153  		t.Fatal("Host is not set!")
   154  	}
   155  
   156  	c := gohbase.NewClient(*host)
   157  	defer c.Close()
   158  	err := insertKeyValue(c, key, "cf", val)
   159  	if err != nil {
   160  		t.Errorf("Put returned an error: %v", err)
   161  	}
   162  
   163  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
   164  	if err != nil {
   165  		t.Fatalf("Failed to create Get request: %s", err)
   166  	}
   167  	rsp, err := c.Get(get)
   168  	if err != nil {
   169  		t.Errorf("Get returned an error: %v", err)
   170  	}
   171  	rsp_value := rsp.Cells[0].Value
   172  	if !bytes.Equal(rsp_value, val) {
   173  		t.Errorf("Get returned an incorrect result. Expected: %v, Got: %v",
   174  			val, rsp_value)
   175  	}
   176  
   177  	get.ExistsOnly()
   178  	rsp, err = c.Get(get)
   179  	if err != nil {
   180  		t.Errorf("Get returned an error: %v", err)
   181  	} else if !*rsp.Exists {
   182  		t.Error("Get claimed that our row didn't exist")
   183  	}
   184  
   185  	ctx, _ := context.WithTimeout(context.Background(), 0)
   186  	get, err = hrpc.NewGetStr(ctx, table, key, hrpc.Families(headers))
   187  	if err != nil {
   188  		t.Fatalf("Failed to create Get request: %s", err)
   189  	}
   190  	_, err = c.Get(get)
   191  	if err != context.DeadlineExceeded {
   192  		t.Errorf("Get ignored the deadline")
   193  	}
   194  }
   195  
   196  func TestGetDoesntExist(t *testing.T) {
   197  	key := "row1.5"
   198  	c := gohbase.NewClient(*host)
   199  	defer c.Close()
   200  	headers := map[string][]string{"cf": nil}
   201  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
   202  	rsp, err := c.Get(get)
   203  	if err != nil {
   204  		t.Errorf("Get returned an error: %v", err)
   205  	} else if results := len(rsp.Cells); results != 0 {
   206  		t.Errorf("Get expected 0 cells. Received: %d", results)
   207  	}
   208  
   209  	get.ExistsOnly()
   210  	rsp, err = c.Get(get)
   211  	if err != nil {
   212  		t.Errorf("Get returned an error: %v", err)
   213  	} else if *rsp.Exists {
   214  		t.Error("Get claimed that our non-existent row exists")
   215  	}
   216  }
   217  
   218  func TestMutateGetTableNotFound(t *testing.T) {
   219  	c := gohbase.NewClient(*host)
   220  	defer c.Close()
   221  
   222  	key := "whatever"
   223  	table := "NonExistentTable"
   224  	headers := map[string][]string{"cf": nil}
   225  	get, err := hrpc.NewGetStr(context.Background(),
   226  		table, key, hrpc.Families(headers))
   227  	if err != nil {
   228  		t.Fatalf("NewGetStr returned an error: %v", err)
   229  	}
   230  	_, err = c.Get(get)
   231  	if err != gohbase.TableNotFound {
   232  		t.Errorf("Get returned unexpected error: %v", err)
   233  	}
   234  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}}
   235  	putRequest, err := hrpc.NewPutStr(context.Background(), table, key, values)
   236  	if err != nil {
   237  		t.Fatalf("NewPutStr returned an error: %v", err)
   238  	}
   239  	_, err = c.Put(putRequest)
   240  	if err != gohbase.TableNotFound {
   241  		t.Errorf("Put returned an unexpected error: %v", err)
   242  	}
   243  }
   244  
   245  func TestGetBadColumnFamily(t *testing.T) {
   246  	key := "row1.625"
   247  	c := gohbase.NewClient(*host)
   248  	defer c.Close()
   249  	err := insertKeyValue(c, key, "cf", []byte("Bad!"))
   250  	if err != nil {
   251  		t.Errorf("Put returned an error: %v", err)
   252  	}
   253  	families := map[string][]string{"badcf": nil}
   254  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(families))
   255  	rsp, err := c.Get(get)
   256  	if err == nil {
   257  		t.Errorf("Get didn't return an error! (It should have)")
   258  	}
   259  	if rsp != nil {
   260  		t.Errorf("Get expected no result. Received: %v", rsp)
   261  	}
   262  }
   263  
   264  func TestGetMultipleCells(t *testing.T) {
   265  	key := "row1.75"
   266  	c := gohbase.NewClient(*host, gohbase.FlushInterval(time.Millisecond*2))
   267  	defer c.Close()
   268  	err := insertKeyValue(c, key, "cf", []byte("cf"))
   269  	if err != nil {
   270  		t.Errorf("Put returned an error: %v", err)
   271  	}
   272  	err = insertKeyValue(c, key, "cf2", []byte("cf2"))
   273  	if err != nil {
   274  		t.Errorf("Put returned an error: %v", err)
   275  	}
   276  
   277  	families := map[string][]string{"cf": nil, "cf2": nil}
   278  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(families))
   279  	rsp, err := c.Get(get)
   280  	cells := rsp.Cells
   281  	num_results := len(cells)
   282  	if num_results != 2 {
   283  		t.Errorf("Get expected 2 cells. Received: %d", num_results)
   284  	}
   285  	for _, cell := range cells {
   286  		if !bytes.Equal(cell.Family, cell.Value) {
   287  			t.Errorf("Get returned an incorrect result. Expected: %v, Received: %v",
   288  				cell.Family, cell.Value)
   289  		}
   290  	}
   291  }
   292  
   293  func TestGetNonDefaultNamespace(t *testing.T) {
   294  	c := gohbase.NewClient(*host)
   295  	defer c.Close()
   296  
   297  	get, err := hrpc.NewGetStr(context.Background(), "hbase:namespace", "default")
   298  	if err != nil {
   299  		t.Fatalf("Failed to create Get request: %s", err)
   300  	}
   301  	rsp, err := c.Get(get)
   302  	if err != nil {
   303  		t.Fatalf("Get returned an error: %v", err)
   304  	}
   305  	if !bytes.Equal(rsp.Cells[0].Family, []byte("info")) {
   306  		t.Errorf("Got unexpected column family: %q", rsp.Cells[0].Family)
   307  	}
   308  }
   309  
   310  func TestPut(t *testing.T) {
   311  	values := map[string]map[string][]byte{"cf": {"a": []byte("1")}}
   312  	if host == nil {
   313  		t.Fatal("Host is not set!")
   314  	}
   315  
   316  	expectNoErr := func(t *testing.T, err error) {}
   317  
   318  	tests := []struct {
   319  		name      string
   320  		keylen    int
   321  		expectErr func(t *testing.T, err error)
   322  	}{
   323  		{
   324  			name:      "Normal",
   325  			keylen:    10,
   326  			expectErr: nil,
   327  		},
   328  		{
   329  			// Test that we can insert a row of len = MAX_ROW_LENGTH
   330  			name:      "MaxRowLength",
   331  			keylen:    math.MaxInt16,
   332  			expectErr: expectNoErr,
   333  		},
   334  		{
   335  			name:   "RowTooLong",
   336  			keylen: math.MaxInt16 + 1,
   337  			expectErr: func(t *testing.T, err error) {
   338  				javaException := "java.io.IOException: Row length 32768 is > 32767"
   339  				if err != nil && strings.Contains(err.Error(), javaException) {
   340  					return
   341  				}
   342  				t.Errorf("expected err=%q, got err=%v", javaException, err)
   343  			},
   344  		},
   345  	}
   346  
   347  	for _, tc := range tests {
   348  		t.Run(tc.name, func(t *testing.T) {
   349  			// It is important for some of these tests to be *100% isolated* to verify that we can
   350  			// first locate the region and then perform the put.
   351  			c := gohbase.NewClient(*host)
   352  			defer c.Close()
   353  
   354  			key := make([]byte, tc.keylen)
   355  			_, err := rand.Read(key)
   356  			if err != nil {
   357  				t.Fatalf("Failed to generate random key: %v", err)
   358  			}
   359  
   360  			putRequest, err := hrpc.NewPut(context.Background(), []byte(table), key, values)
   361  			if err != nil {
   362  				t.Errorf("NewPutStr returned an error: %v", err)
   363  			}
   364  			_, err = c.Put(putRequest)
   365  			expectNoErr(t, err)
   366  		})
   367  	}
   368  }
   369  
   370  func TestPutWithTimeout(t *testing.T) {
   371  	key := "row2"
   372  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}}
   373  	if host == nil {
   374  		t.Fatal("Host is not set!")
   375  	}
   376  	c := gohbase.NewClient(*host)
   377  	defer c.Close()
   378  
   379  	ctx, _ := context.WithTimeout(context.Background(), 0)
   380  	putRequest, err := hrpc.NewPutStr(ctx, table, key, values)
   381  	_, err = c.Put(putRequest)
   382  	if err != context.DeadlineExceeded {
   383  		t.Errorf("Put ignored the deadline")
   384  	}
   385  }
   386  
   387  func TestPutMultipleCells(t *testing.T) {
   388  	key := "row2.5"
   389  	values := map[string]map[string][]byte{"cf": map[string][]byte{}, "cf2": map[string][]byte{}}
   390  	values["cf"]["a"] = []byte("a")
   391  	values["cf"]["b"] = []byte("b")
   392  	values["cf2"]["a"] = []byte("a")
   393  	c := gohbase.NewClient(*host)
   394  	defer c.Close()
   395  	putRequest, err := hrpc.NewPutStr(context.Background(), table, key, values)
   396  	_, err = c.Put(putRequest)
   397  	if err != nil {
   398  		t.Errorf("Put returned an error: %v", err)
   399  	}
   400  	families := map[string][]string{"cf": nil, "cf2": nil}
   401  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(families))
   402  	rsp, err := c.Get(get)
   403  	if err != nil {
   404  		t.Errorf("Get returned an error: %v", err)
   405  	}
   406  	cells := rsp.Cells
   407  	if len(cells) != 3 {
   408  		t.Errorf("Get expected 3 cells. Received: %d", len(cells))
   409  	}
   410  	for _, cell := range cells {
   411  		if !bytes.Equal(cell.Qualifier, cell.Value) {
   412  			t.Errorf("Get returned an incorrect result. Expected: %v, Received: %v",
   413  				cell.Qualifier, cell.Value)
   414  		}
   415  	}
   416  
   417  }
   418  
   419  func TestMultiplePutsGetsSequentially(t *testing.T) {
   420  	const num_ops = 100
   421  	keyPrefix := "row3"
   422  	headers := map[string][]string{"cf": nil}
   423  	c := gohbase.NewClient(*host, gohbase.FlushInterval(time.Millisecond))
   424  	defer c.Close()
   425  	err := performNPuts(keyPrefix, num_ops)
   426  	if err != nil {
   427  		t.Errorf("Put returned an error: %v", err)
   428  	}
   429  	for i := num_ops - 1; i >= 0; i-- {
   430  		key := keyPrefix + fmt.Sprintf("%d", i)
   431  		get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
   432  		rsp, err := c.Get(get)
   433  		if err != nil {
   434  			t.Errorf("Get returned an error: %v", err)
   435  		}
   436  		if len(rsp.Cells) != 1 {
   437  			t.Errorf("Incorrect number of cells returned by Get: %d", len(rsp.Cells))
   438  		}
   439  		rsp_value := rsp.Cells[0].Value
   440  		if !bytes.Equal(rsp_value, []byte(fmt.Sprintf("%d", i))) {
   441  			t.Errorf("Get returned an incorrect result. Expected: %v, Got: %v",
   442  				[]byte(fmt.Sprintf("%d", i)), rsp_value)
   443  		}
   444  	}
   445  }
   446  
   447  func TestMultiplePutsGetsParallel(t *testing.T) {
   448  	c := gohbase.NewClient(*host)
   449  	defer c.Close()
   450  
   451  	const n = 1000
   452  	var wg sync.WaitGroup
   453  	for i := 0; i < n; i++ {
   454  		key := fmt.Sprintf("%s_%d", t.Name(), i)
   455  		wg.Add(1)
   456  		go func() {
   457  			if err := insertKeyValue(c, key, "cf", []byte(key)); err != nil {
   458  				t.Error(key, err)
   459  			}
   460  			wg.Done()
   461  		}()
   462  	}
   463  	wg.Wait()
   464  
   465  	// All puts are complete. Now do the same for gets.
   466  	headers := map[string][]string{"cf": []string{"a"}}
   467  	for i := n - 1; i >= 0; i-- {
   468  		key := fmt.Sprintf("%s_%d", t.Name(), i)
   469  		wg.Add(1)
   470  		go func() {
   471  			defer wg.Done()
   472  			get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
   473  			if err != nil {
   474  				t.Error(key, err)
   475  				return
   476  			}
   477  			rsp, err := c.Get(get)
   478  			if err != nil {
   479  				t.Error(key, err)
   480  				return
   481  			}
   482  			if len(rsp.Cells) == 0 {
   483  				t.Error(key, " got zero cells")
   484  				return
   485  			}
   486  			rsp_value := rsp.Cells[0].Value
   487  			if !bytes.Equal(rsp_value, []byte(key)) {
   488  				t.Errorf("expected %q, got %q", key, rsp_value)
   489  			}
   490  		}()
   491  	}
   492  	wg.Wait()
   493  }
   494  
   495  func TestTimestampIncreasing(t *testing.T) {
   496  	key := "row4"
   497  	c := gohbase.NewClient(*host)
   498  	defer c.Close()
   499  	var oldTime uint64 = 0
   500  	headers := map[string][]string{"cf": nil}
   501  	for i := 0; i < 10; i++ {
   502  		insertKeyValue(c, key, "cf", []byte("1"))
   503  		get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
   504  		rsp, err := c.Get(get)
   505  		if err != nil {
   506  			t.Errorf("Get returned an error: %v", err)
   507  			break
   508  		}
   509  		newTime := *rsp.Cells[0].Timestamp
   510  		if newTime <= oldTime {
   511  			t.Errorf("Timestamps are not increasing. Old Time: %v, New Time: %v",
   512  				oldTime, newTime)
   513  		}
   514  		oldTime = newTime
   515  		time.Sleep(time.Millisecond)
   516  	}
   517  }
   518  
   519  func TestPutTimestamp(t *testing.T) {
   520  	key := "TestPutTimestamp"
   521  	c := gohbase.NewClient(*host)
   522  	defer c.Close()
   523  	var putTs uint64 = 50
   524  	timestamp := time.Unix(0, int64(putTs*1e6))
   525  	err := insertKeyValue(c, key, "cf", []byte("1"), hrpc.Timestamp(timestamp))
   526  	if err != nil {
   527  		t.Fatalf("Put failed: %s", err)
   528  	}
   529  	get, err := hrpc.NewGetStr(context.Background(), table, key,
   530  		hrpc.Families(map[string][]string{"cf": nil}))
   531  	rsp, err := c.Get(get)
   532  	if err != nil {
   533  		t.Fatalf("Get failed: %s", err)
   534  	}
   535  	getTs := *rsp.Cells[0].Timestamp
   536  	if getTs != putTs {
   537  		t.Errorf("Timestamps are not the same. Put Time: %v, Get Time: %v",
   538  			putTs, getTs)
   539  	}
   540  }
   541  
   542  // TestDelete preps state with two column families, cf1 and cf2,
   543  // each having 3 versions at timestamps 50, 51, 52
   544  func TestDelete(t *testing.T) {
   545  	c := gohbase.NewClient(*host)
   546  	defer c.Close()
   547  
   548  	ts := uint64(50)
   549  
   550  	tests := []struct {
   551  		in  func(string) (*hrpc.Mutate, error)
   552  		out []*hrpc.Cell
   553  	}{
   554  		{
   555  			// delete at the second version
   556  			in: func(key string) (*hrpc.Mutate, error) {
   557  				return hrpc.NewDelStr(context.Background(), table, key,
   558  					map[string]map[string][]byte{"cf1": map[string][]byte{"a": nil}},
   559  					hrpc.TimestampUint64(ts+1))
   560  			},
   561  			// should delete everything at and before the delete timestamp
   562  			out: []*hrpc.Cell{
   563  				&hrpc.Cell{
   564  					Family:    []byte("cf1"),
   565  					Qualifier: []byte("a"),
   566  					Timestamp: proto.Uint64(ts + 2),
   567  					Value:     []byte("v3"),
   568  				},
   569  				&hrpc.Cell{
   570  					Family:    []byte("cf1"),
   571  					Qualifier: []byte("b"),
   572  					Timestamp: proto.Uint64(ts),
   573  					Value:     []byte("v1"),
   574  				},
   575  				&hrpc.Cell{
   576  					Family:    []byte("cf2"),
   577  					Qualifier: []byte("a"),
   578  					Timestamp: proto.Uint64(ts + 2),
   579  					Value:     []byte("v3"),
   580  				},
   581  				&hrpc.Cell{
   582  					Family:    []byte("cf2"),
   583  					Qualifier: []byte("a"),
   584  					Timestamp: proto.Uint64(ts + 1),
   585  					Value:     []byte("v2"),
   586  				},
   587  				&hrpc.Cell{
   588  					Family:    []byte("cf2"),
   589  					Qualifier: []byte("a"),
   590  					Timestamp: proto.Uint64(ts),
   591  					Value:     []byte("v1"),
   592  				},
   593  				&hrpc.Cell{
   594  					Family:    []byte("cf2"),
   595  					Qualifier: []byte("b"),
   596  					Timestamp: proto.Uint64(ts),
   597  					Value:     []byte("v1"),
   598  				},
   599  			},
   600  		},
   601  		{
   602  			// delete at the second version
   603  			in: func(key string) (*hrpc.Mutate, error) {
   604  				return hrpc.NewDelStr(context.Background(), table, key,
   605  					map[string]map[string][]byte{"cf1": map[string][]byte{"a": nil}},
   606  					hrpc.TimestampUint64(ts+1), hrpc.DeleteOneVersion())
   607  			},
   608  			// should delete only the second version
   609  			out: []*hrpc.Cell{
   610  				&hrpc.Cell{
   611  					Family:    []byte("cf1"),
   612  					Qualifier: []byte("a"),
   613  					Timestamp: proto.Uint64(ts + 2),
   614  					Value:     []byte("v3"),
   615  				},
   616  				&hrpc.Cell{
   617  					Family:    []byte("cf1"),
   618  					Qualifier: []byte("a"),
   619  					Timestamp: proto.Uint64(ts),
   620  					Value:     []byte("v1"),
   621  				},
   622  				&hrpc.Cell{
   623  					Family:    []byte("cf1"),
   624  					Qualifier: []byte("b"),
   625  					Timestamp: proto.Uint64(ts),
   626  					Value:     []byte("v1"),
   627  				},
   628  				&hrpc.Cell{
   629  					Family:    []byte("cf2"),
   630  					Qualifier: []byte("a"),
   631  					Timestamp: proto.Uint64(ts + 2),
   632  					Value:     []byte("v3"),
   633  				},
   634  				&hrpc.Cell{
   635  					Family:    []byte("cf2"),
   636  					Qualifier: []byte("a"),
   637  					Timestamp: proto.Uint64(ts + 1),
   638  					Value:     []byte("v2"),
   639  				},
   640  				&hrpc.Cell{
   641  					Family:    []byte("cf2"),
   642  					Qualifier: []byte("a"),
   643  					Timestamp: proto.Uint64(ts),
   644  					Value:     []byte("v1"),
   645  				},
   646  				&hrpc.Cell{
   647  					Family:    []byte("cf2"),
   648  					Qualifier: []byte("b"),
   649  					Timestamp: proto.Uint64(ts),
   650  					Value:     []byte("v1"),
   651  				},
   652  			},
   653  		},
   654  		{
   655  			// delete the cf1 at and before ts + 1
   656  			in: func(key string) (*hrpc.Mutate, error) {
   657  				return hrpc.NewDelStr(context.Background(), table, key,
   658  					map[string]map[string][]byte{"cf1": nil},
   659  					hrpc.TimestampUint64(ts+1))
   660  			},
   661  			// should leave cf2 untouched
   662  			out: []*hrpc.Cell{
   663  				&hrpc.Cell{
   664  					Family:    []byte("cf1"),
   665  					Qualifier: []byte("a"),
   666  					Timestamp: proto.Uint64(ts + 2),
   667  					Value:     []byte("v3"),
   668  				},
   669  				&hrpc.Cell{
   670  					Family:    []byte("cf2"),
   671  					Qualifier: []byte("a"),
   672  					Timestamp: proto.Uint64(ts + 2),
   673  					Value:     []byte("v3"),
   674  				},
   675  				&hrpc.Cell{
   676  					Family:    []byte("cf2"),
   677  					Qualifier: []byte("a"),
   678  					Timestamp: proto.Uint64(ts + 1),
   679  					Value:     []byte("v2"),
   680  				},
   681  				&hrpc.Cell{
   682  					Family:    []byte("cf2"),
   683  					Qualifier: []byte("a"),
   684  					Timestamp: proto.Uint64(ts),
   685  					Value:     []byte("v1"),
   686  				},
   687  				&hrpc.Cell{
   688  					Family:    []byte("cf2"),
   689  					Qualifier: []byte("b"),
   690  					Timestamp: proto.Uint64(ts),
   691  					Value:     []byte("v1"),
   692  				},
   693  			},
   694  		},
   695  		{
   696  			// delete the whole cf1 and qualifer a in cf2
   697  			in: func(key string) (*hrpc.Mutate, error) {
   698  				return hrpc.NewDelStr(context.Background(), table, key,
   699  					map[string]map[string][]byte{
   700  						"cf1": nil,
   701  						"cf2": map[string][]byte{
   702  							"a": nil,
   703  						},
   704  					})
   705  			},
   706  			out: []*hrpc.Cell{
   707  				&hrpc.Cell{
   708  					Family:    []byte("cf2"),
   709  					Qualifier: []byte("b"),
   710  					Timestamp: proto.Uint64(ts),
   711  					Value:     []byte("v1"),
   712  				},
   713  			},
   714  		},
   715  		{
   716  			// delete only version at ts for all qualifiers of cf1
   717  			in: func(key string) (*hrpc.Mutate, error) {
   718  				return hrpc.NewDelStr(context.Background(), table, key,
   719  					map[string]map[string][]byte{
   720  						"cf1": nil,
   721  					}, hrpc.TimestampUint64(ts), hrpc.DeleteOneVersion())
   722  			},
   723  			out: []*hrpc.Cell{
   724  				&hrpc.Cell{
   725  					Family:    []byte("cf1"),
   726  					Qualifier: []byte("a"),
   727  					Timestamp: proto.Uint64(ts + 2),
   728  					Value:     []byte("v3"),
   729  				},
   730  				&hrpc.Cell{
   731  					Family:    []byte("cf1"),
   732  					Qualifier: []byte("a"),
   733  					Timestamp: proto.Uint64(ts + 1),
   734  					Value:     []byte("v2"),
   735  				},
   736  				&hrpc.Cell{
   737  					Family:    []byte("cf2"),
   738  					Qualifier: []byte("a"),
   739  					Timestamp: proto.Uint64(ts + 2),
   740  					Value:     []byte("v3"),
   741  				},
   742  				&hrpc.Cell{
   743  					Family:    []byte("cf2"),
   744  					Qualifier: []byte("a"),
   745  					Timestamp: proto.Uint64(ts + 1),
   746  					Value:     []byte("v2"),
   747  				},
   748  				&hrpc.Cell{
   749  					Family:    []byte("cf2"),
   750  					Qualifier: []byte("a"),
   751  					Timestamp: proto.Uint64(ts),
   752  					Value:     []byte("v1"),
   753  				},
   754  				&hrpc.Cell{
   755  					Family:    []byte("cf2"),
   756  					Qualifier: []byte("b"),
   757  					Timestamp: proto.Uint64(ts),
   758  					Value:     []byte("v1"),
   759  				},
   760  			},
   761  		},
   762  		{
   763  			// delete the whole row
   764  			in: func(key string) (*hrpc.Mutate, error) {
   765  				return hrpc.NewDelStr(context.Background(), table, key, nil)
   766  			},
   767  			out: nil,
   768  		},
   769  		{
   770  			// delete the whole row at ts
   771  			in: func(key string) (*hrpc.Mutate, error) {
   772  				return hrpc.NewDelStr(context.Background(), table, key, nil,
   773  					hrpc.TimestampUint64(ts+1))
   774  			},
   775  			out: []*hrpc.Cell{
   776  				&hrpc.Cell{
   777  					Family:    []byte("cf1"),
   778  					Qualifier: []byte("a"),
   779  					Timestamp: proto.Uint64(ts + 2),
   780  					Value:     []byte("v3"),
   781  				},
   782  				&hrpc.Cell{
   783  					Family:    []byte("cf2"),
   784  					Qualifier: []byte("a"),
   785  					Timestamp: proto.Uint64(ts + 2),
   786  					Value:     []byte("v3"),
   787  				},
   788  			},
   789  		},
   790  	}
   791  
   792  	for i, tcase := range tests {
   793  		t.Run(strconv.Itoa(i), func(t *testing.T) {
   794  			key := t.Name()
   795  
   796  			// insert three versions
   797  			prep := func(cf string) {
   798  				if err := insertKeyValue(c, key, cf, []byte("v1"),
   799  					hrpc.TimestampUint64(ts)); err != nil {
   800  					t.Fatal(err)
   801  				}
   802  
   803  				if err := insertKeyValue(c, key, cf, []byte("v2"),
   804  					hrpc.TimestampUint64(ts+1)); err != nil {
   805  					t.Fatal(err)
   806  				}
   807  
   808  				if err := insertKeyValue(c, key, cf, []byte("v3"),
   809  					hrpc.TimestampUint64(ts+2)); err != nil {
   810  					t.Fatal(err)
   811  				}
   812  
   813  				// insert b
   814  				values := map[string]map[string][]byte{cf: map[string][]byte{
   815  					"b": []byte("v1"),
   816  				}}
   817  				put, err := hrpc.NewPutStr(context.Background(), table, key, values,
   818  					hrpc.TimestampUint64(ts))
   819  				if err != nil {
   820  					t.Fatal(err)
   821  				}
   822  				if _, err = c.Put(put); err != nil {
   823  					t.Fatal(err)
   824  				}
   825  			}
   826  
   827  			prep("cf1")
   828  			prep("cf2")
   829  
   830  			delete, err := tcase.in(key)
   831  			if err != nil {
   832  				t.Fatal(err)
   833  			}
   834  
   835  			_, err = c.Delete(delete)
   836  			if err != nil {
   837  				t.Fatal(err)
   838  			}
   839  
   840  			get, err := hrpc.NewGetStr(context.Background(), table, key,
   841  				hrpc.MaxVersions(math.MaxInt32))
   842  			if err != nil {
   843  				t.Fatal(err)
   844  			}
   845  
   846  			rsp, err := c.Get(get)
   847  			if err != nil {
   848  				t.Fatal(err)
   849  			}
   850  
   851  			for _, c := range tcase.out {
   852  				c.Row = []byte(t.Name())
   853  				c.CellType = pb.CellType_PUT.Enum()
   854  			}
   855  
   856  			if !reflect.DeepEqual(tcase.out, rsp.Cells) {
   857  				t.Fatalf("expected %v, got %v", tcase.out, rsp.Cells)
   858  			}
   859  		})
   860  	}
   861  }
   862  
   863  func TestGetTimeRangeVersions(t *testing.T) {
   864  	key := "TestGetTimeRangeVersions"
   865  	c := gohbase.NewClient(*host)
   866  	defer c.Close()
   867  	err := insertKeyValue(c, key, "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 50*1e6)))
   868  	if err != nil {
   869  		t.Fatalf("Put failed: %s", err)
   870  	}
   871  	err = insertKeyValue(c, key, "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 51*1e6)))
   872  	if err != nil {
   873  		t.Fatalf("Put failed: %s", err)
   874  	}
   875  	err = insertKeyValue(c, key, "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 49*1e6)))
   876  	if err != nil {
   877  		t.Fatalf("Put failed: %s", err)
   878  	}
   879  
   880  	var maxVersions uint32 = 2
   881  	get, err := hrpc.NewGetStr(context.Background(), table, key,
   882  		hrpc.Families(map[string][]string{"cf": nil}), hrpc.TimeRange(time.Unix(0, 0),
   883  			time.Unix(0, 51*1e6)), hrpc.MaxVersions(maxVersions))
   884  	rsp, err := c.Get(get)
   885  	if err != nil {
   886  		t.Fatalf("Get failed: %s", err)
   887  	}
   888  	if uint32(len(rsp.Cells)) != maxVersions {
   889  		t.Fatalf("Expected versions: %d, Got versions: %d", maxVersions, len(rsp.Cells))
   890  	}
   891  	getTs1 := *rsp.Cells[0].Timestamp
   892  	if getTs1 != 50 {
   893  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   894  			50, getTs1)
   895  	}
   896  	getTs2 := *rsp.Cells[1].Timestamp
   897  	if getTs2 != 49 {
   898  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   899  			49, getTs2)
   900  	}
   901  
   902  	// get with no versions set
   903  	get, err = hrpc.NewGetStr(context.Background(), table, key,
   904  		hrpc.Families(map[string][]string{"cf": nil}), hrpc.TimeRange(time.Unix(0, 0),
   905  			time.Unix(0, 51*1e6)))
   906  	rsp, err = c.Get(get)
   907  	if err != nil {
   908  		t.Fatalf("Get failed: %s", err)
   909  	}
   910  	if uint32(len(rsp.Cells)) != 1 {
   911  		t.Fatalf("Expected versions: %d, Got versions: %d", 1, len(rsp.Cells))
   912  	}
   913  	getTs1 = *rsp.Cells[0].Timestamp
   914  	if getTs1 != 50 {
   915  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   916  			50, getTs1)
   917  	}
   918  }
   919  
   920  func TestScanTimeRangeVersions(t *testing.T) {
   921  	key := "TestScanTimeRangeVersions"
   922  	c := gohbase.NewClient(*host)
   923  	defer c.Close()
   924  	err := insertKeyValue(c, key+"1", "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 50*1e6)))
   925  	if err != nil {
   926  		t.Fatalf("Put failed: %s", err)
   927  	}
   928  	err = insertKeyValue(c, key+"1", "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 51*1e6)))
   929  	if err != nil {
   930  		t.Fatalf("Put failed: %s", err)
   931  	}
   932  	err = insertKeyValue(c, key+"2", "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 51*1e6)))
   933  	if err != nil {
   934  		t.Fatalf("Put failed: %s", err)
   935  	}
   936  	err = insertKeyValue(c, key+"2", "cf", []byte("1"), hrpc.Timestamp(time.Unix(0, 52*1e6)))
   937  	if err != nil {
   938  		t.Fatalf("Put failed: %s", err)
   939  	}
   940  
   941  	var maxVersions uint32 = 2
   942  	scan, err := hrpc.NewScanRangeStr(context.Background(), table,
   943  		"TestScanTimeRangeVersions1", "TestScanTimeRangeVersions3",
   944  		hrpc.Families(map[string][]string{"cf": nil}), hrpc.TimeRange(time.Unix(0, 50*1e6),
   945  			time.Unix(0, 53*1e6)), hrpc.MaxVersions(maxVersions))
   946  	if err != nil {
   947  		t.Fatalf("Scan req failed: %s", err)
   948  	}
   949  
   950  	var rsp []*hrpc.Result
   951  	scanner := c.Scan(scan)
   952  	for {
   953  		res, err := scanner.Next()
   954  		if err == io.EOF {
   955  			break
   956  		}
   957  		if err != nil {
   958  			t.Fatal(err)
   959  		}
   960  		rsp = append(rsp, res)
   961  	}
   962  
   963  	if len(rsp) != 2 {
   964  		t.Fatalf("Expected rows: %d, Got rows: %d", maxVersions, len(rsp))
   965  	}
   966  	if uint32(len(rsp[0].Cells)) != maxVersions {
   967  		t.Fatalf("Expected versions: %d, Got versions: %d", maxVersions, len(rsp[0].Cells))
   968  	}
   969  	scan1 := *rsp[0].Cells[0]
   970  	if string(scan1.Row) != "TestScanTimeRangeVersions1" && *scan1.Timestamp != 51 {
   971  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   972  			51, *scan1.Timestamp)
   973  	}
   974  	scan2 := *rsp[0].Cells[1]
   975  	if string(scan2.Row) != "TestScanTimeRangeVersions1" && *scan2.Timestamp != 50 {
   976  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   977  			50, *scan2.Timestamp)
   978  	}
   979  	if uint32(len(rsp[1].Cells)) != maxVersions {
   980  		t.Fatalf("Expected versions: %d, Got versions: %d", maxVersions, len(rsp[1].Cells))
   981  	}
   982  	scan3 := *rsp[1].Cells[0]
   983  	if string(scan3.Row) != "TestScanTimeRangeVersions2" && *scan3.Timestamp != 52 {
   984  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   985  			52, *scan3.Timestamp)
   986  	}
   987  	scan4 := *rsp[1].Cells[1]
   988  	if string(scan4.Row) != "TestScanTimeRangeVersions2" && *scan4.Timestamp != 51 {
   989  		t.Errorf("Timestamps are not the same. Expected Time: %v, Got Time: %v",
   990  			51, *scan4.Timestamp)
   991  	}
   992  
   993  	// scan with no versions set
   994  	scan, err = hrpc.NewScanRangeStr(context.Background(), table,
   995  		"TestScanTimeRangeVersions1", "TestScanTimeRangeVersions3",
   996  		hrpc.Families(map[string][]string{"cf": nil}), hrpc.TimeRange(time.Unix(0, 50*1e6),
   997  			time.Unix(0, 53*1e6)),
   998  		hrpc.NumberOfRows(1)) // set number of rows to 1 to also check that we are doing fetches
   999  	if err != nil {
  1000  		t.Fatalf("Scan req failed: %s", err)
  1001  	}
  1002  
  1003  	rsp = nil
  1004  	scanner = c.Scan(scan)
  1005  	for {
  1006  		res, err := scanner.Next()
  1007  		if err == io.EOF {
  1008  			break
  1009  		}
  1010  		if err != nil {
  1011  			t.Fatal(err)
  1012  		}
  1013  		rsp = append(rsp, res)
  1014  	}
  1015  	if len(rsp) != 2 {
  1016  		t.Fatalf("Expected rows: %d, Got rows: %d", 2, len(rsp))
  1017  	}
  1018  	if len(rsp[0].Cells) != 1 {
  1019  		t.Fatalf("Expected versions: %d, Got versions: %d", 2, len(rsp[0].Cells))
  1020  	}
  1021  	if len(rsp[1].Cells) != 1 {
  1022  		t.Fatalf("Expected versions: %d, Got versions: %d", 2, len(rsp[0].Cells))
  1023  	}
  1024  }
  1025  
  1026  func TestScanWithScanMetrics(t *testing.T) {
  1027  	var (
  1028  		key          = "TestScanWithScanMetrics"
  1029  		now          = time.Now()
  1030  		r1           = fmt.Sprintf("%s_%d", key, 1)
  1031  		r2           = fmt.Sprintf("%s_%d", key, 2)
  1032  		r3           = fmt.Sprintf("%s_%d", key, 3)
  1033  		val          = []byte("1")
  1034  		family       = "cf"
  1035  		ctx          = context.Background()
  1036  		rowsScanned  = "ROWS_SCANNED"
  1037  		rowsFiltered = "ROWS_FILTERED"
  1038  	)
  1039  
  1040  	c := gohbase.NewClient(*host)
  1041  	defer c.Close()
  1042  
  1043  	for _, r := range []string{r1, r2, r3} {
  1044  		err := insertKeyValue(c, r, family, val, hrpc.Timestamp(now))
  1045  		if err != nil {
  1046  			t.Fatalf("Put failed: %s", err)
  1047  		}
  1048  	}
  1049  
  1050  	tcases := []struct {
  1051  		description          string
  1052  		filters              func(call hrpc.Call) error
  1053  		expectedRowsScanned  int64
  1054  		expectedRowsFiltered int64
  1055  		noScanMetrics        bool
  1056  	}{
  1057  		{
  1058  			description:          "scan metrics not enabled",
  1059  			expectedRowsScanned:  0,
  1060  			expectedRowsFiltered: 0,
  1061  			noScanMetrics:        true,
  1062  		},
  1063  		{
  1064  			description:          "2 rows scanned",
  1065  			expectedRowsScanned:  2,
  1066  			expectedRowsFiltered: 0,
  1067  		},
  1068  		{
  1069  			description:          "1 row scanned 1 row filtered",
  1070  			filters:              hrpc.Filters(filter.NewPrefixFilter([]byte(r1))),
  1071  			expectedRowsScanned:  1,
  1072  			expectedRowsFiltered: 1,
  1073  		},
  1074  	}
  1075  
  1076  	for _, tc := range tcases {
  1077  		t.Run(tc.description, func(t *testing.T) {
  1078  			var (
  1079  				scan *hrpc.Scan
  1080  				err  error
  1081  			)
  1082  			if tc.noScanMetrics {
  1083  				scan, err = hrpc.NewScanRangeStr(ctx, table, r1, r3)
  1084  			} else if tc.filters == nil {
  1085  				scan, err = hrpc.NewScanRangeStr(ctx, table, r1, r3, hrpc.TrackScanMetrics())
  1086  			} else {
  1087  				scan, err = hrpc.NewScanRangeStr(ctx, table, r1, r3, hrpc.TrackScanMetrics(),
  1088  					tc.filters)
  1089  			}
  1090  			if err != nil {
  1091  				t.Fatalf("Scan req failed: %s", err)
  1092  			}
  1093  
  1094  			var results []*hrpc.Result
  1095  			scanner := c.Scan(scan)
  1096  			for {
  1097  				var r *hrpc.Result
  1098  				r, err = scanner.Next()
  1099  				if err == io.EOF {
  1100  					break
  1101  				}
  1102  				if err != nil {
  1103  					t.Fatal(err)
  1104  				}
  1105  				results = append(results, r)
  1106  			}
  1107  
  1108  			actualMetrics := scanner.GetScanMetrics()
  1109  
  1110  			if tc.noScanMetrics && actualMetrics != nil {
  1111  				t.Fatalf("Expected nil scan metrics, got %v", actualMetrics)
  1112  			}
  1113  
  1114  			scanned := actualMetrics[rowsScanned]
  1115  			if tc.expectedRowsScanned != scanned {
  1116  				t.Errorf("Did not get expected rows scanned - expected: %d, actual %d",
  1117  					tc.expectedRowsScanned, scanned)
  1118  			}
  1119  
  1120  			filtered := actualMetrics[rowsFiltered]
  1121  			if tc.expectedRowsFiltered != filtered {
  1122  				t.Errorf("Did not get expected rows filtered - expected: %d, actual %d",
  1123  					tc.expectedRowsFiltered, filtered)
  1124  			}
  1125  		})
  1126  	}
  1127  
  1128  }
  1129  
  1130  func TestPutTTL(t *testing.T) {
  1131  	key := "TestPutTTL"
  1132  	c := gohbase.NewClient(*host)
  1133  	defer c.Close()
  1134  
  1135  	var ttl = 2 * time.Second
  1136  
  1137  	err := insertKeyValue(c, key, "cf", []byte("1"), hrpc.TTL(ttl))
  1138  	if err != nil {
  1139  		t.Fatalf("Put failed: %s", err)
  1140  	}
  1141  
  1142  	// Wait ttl duration and try to get the value
  1143  	time.Sleep(ttl)
  1144  
  1145  	get, err := hrpc.NewGetStr(context.Background(), table, key,
  1146  		hrpc.Families(map[string][]string{"cf": nil}))
  1147  
  1148  	// Make sure we don't get a result back
  1149  	res, err := c.Get(get)
  1150  	if err != nil {
  1151  		t.Fatalf("Get failed: %s", err)
  1152  	}
  1153  
  1154  	if len(res.Cells) > 0 {
  1155  		t.Errorf("TTL did not expire row. Expected 0 cells, got: %d", len(res.Cells))
  1156  	}
  1157  }
  1158  
  1159  func checkResultRow(t *testing.T, res *hrpc.Result, expectedRow string, err, expectedErr error) {
  1160  	if err != expectedErr {
  1161  		t.Fatalf("Expected error %v, got error %v", expectedErr, err)
  1162  	}
  1163  	if len(expectedRow) > 0 && res != nil && len(res.Cells) > 0 {
  1164  		got := string(res.Cells[0].Row)
  1165  		if got != expectedRow {
  1166  			t.Fatalf("Expected row %s, got row %s", expectedRow, got)
  1167  		}
  1168  	} else if len(expectedRow) == 0 && res != nil {
  1169  		t.Fatalf("Expected no result, got %+v", *res)
  1170  	}
  1171  }
  1172  
  1173  func TestScannerClose(t *testing.T) {
  1174  	key := t.Name()
  1175  	c := gohbase.NewClient(*host)
  1176  	defer c.Close()
  1177  
  1178  	err := insertKeyValue(c, key+"1", "cf", []byte("1"))
  1179  	if err != nil {
  1180  		t.Fatalf("Put failed: %s", err)
  1181  	}
  1182  	err = insertKeyValue(c, key+"2", "cf", []byte("1"))
  1183  	if err != nil {
  1184  		t.Fatalf("Put failed: %s", err)
  1185  	}
  1186  	err = insertKeyValue(c, key+"3", "cf", []byte("1"))
  1187  	if err != nil {
  1188  		t.Fatalf("Put failed: %s", err)
  1189  	}
  1190  
  1191  	scan, err := hrpc.NewScanRangeStr(context.Background(), table,
  1192  		key+"1", key+"4",
  1193  		hrpc.Families(map[string][]string{"cf": nil}),
  1194  		hrpc.NumberOfRows(1)) // fetch only one row at a time
  1195  	if err != nil {
  1196  		t.Fatalf("Scan req failed: %s", err)
  1197  	}
  1198  	scanner := c.Scan(scan)
  1199  	res, err := scanner.Next()
  1200  	checkResultRow(t, res, key+"1", err, nil)
  1201  
  1202  	res, err = scanner.Next()
  1203  	checkResultRow(t, res, key+"2", err, nil)
  1204  
  1205  	scanner.Close()
  1206  
  1207  	// make sure we get io.EOF eventually
  1208  	for {
  1209  		if _, err = scanner.Next(); err == io.EOF {
  1210  			break
  1211  		}
  1212  	}
  1213  }
  1214  
  1215  func TestScannerContextCancel(t *testing.T) {
  1216  	key := t.Name()
  1217  	c := gohbase.NewClient(*host)
  1218  	defer c.Close()
  1219  
  1220  	err := insertKeyValue(c, key+"1", "cf", []byte("1"))
  1221  	if err != nil {
  1222  		t.Fatalf("Put failed: %s", err)
  1223  	}
  1224  	err = insertKeyValue(c, key+"2", "cf", []byte("1"))
  1225  	if err != nil {
  1226  		t.Fatalf("Put failed: %s", err)
  1227  	}
  1228  	err = insertKeyValue(c, key+"3", "cf", []byte("1"))
  1229  	if err != nil {
  1230  		t.Fatalf("Put failed: %s", err)
  1231  	}
  1232  
  1233  	ctx, cancel := context.WithCancel(context.Background())
  1234  	scan, err := hrpc.NewScanRangeStr(ctx, table,
  1235  		key+"1", key+"4",
  1236  		hrpc.Families(map[string][]string{"cf": nil}),
  1237  		hrpc.NumberOfRows(1)) // fetch only one row at a time
  1238  	if err != nil {
  1239  		t.Fatalf("Scan req failed: %s", err)
  1240  	}
  1241  	scanner := c.Scan(scan)
  1242  	res, err := scanner.Next()
  1243  	checkResultRow(t, res, key+"1", err, nil)
  1244  
  1245  	cancel()
  1246  
  1247  	if _, err = scanner.Next(); err != context.Canceled {
  1248  		t.Fatalf("unexpected error %v, expected %v", err, context.Canceled)
  1249  	}
  1250  }
  1251  
  1252  func TestAppend(t *testing.T) {
  1253  	key := "row7"
  1254  	c := gohbase.NewClient(*host)
  1255  	defer c.Close()
  1256  	// Inserting "Hello"
  1257  	insertErr := insertKeyValue(c, key, "cf", []byte("Hello"))
  1258  	if insertErr != nil {
  1259  		t.Errorf("Put returned an error: %v", insertErr)
  1260  	}
  1261  	// Appending " my name is Dog."
  1262  	values := map[string]map[string][]byte{"cf": map[string][]byte{}}
  1263  	values["cf"]["a"] = []byte(" my name is Dog.")
  1264  	appRequest, err := hrpc.NewAppStr(context.Background(), table, key, values)
  1265  	appRsp, err := c.Append(appRequest)
  1266  	if err != nil {
  1267  		t.Errorf("Append returned an error: %v", err)
  1268  	}
  1269  	if appRsp == nil {
  1270  		t.Errorf("Append doesn't return updated value.")
  1271  	}
  1272  	// Verifying new result is "Hello my name is Dog."
  1273  	result := appRsp.Cells[0].Value
  1274  	if !bytes.Equal([]byte("Hello my name is Dog."), result) {
  1275  		t.Errorf("Append returned an incorrect result. Expected: %v, Receieved: %v",
  1276  			[]byte("Hello my name is Dog."), result)
  1277  	}
  1278  
  1279  	// Make sure the change was actually committed.
  1280  	headers := map[string][]string{"cf": nil}
  1281  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
  1282  	rsp, err := c.Get(get)
  1283  	cells := rsp.Cells
  1284  	if len(cells) != 1 {
  1285  		t.Errorf("Get expected 1 cells. Received: %d", len(cells))
  1286  	}
  1287  	result = cells[0].Value
  1288  	if !bytes.Equal([]byte("Hello my name is Dog."), result) {
  1289  		t.Errorf("Append returned an incorrect result. Expected: %v, Receieved: %v",
  1290  			[]byte("Hello my name is Dog."), result)
  1291  	}
  1292  }
  1293  
  1294  func TestIncrement(t *testing.T) {
  1295  	c := gohbase.NewClient(*host)
  1296  	defer c.Close()
  1297  	key := "row102"
  1298  
  1299  	// test incerement
  1300  	incRequest, err := hrpc.NewIncStrSingle(context.Background(), table, key, "cf", "a", 1)
  1301  	result, err := c.Increment(incRequest)
  1302  	if err != nil {
  1303  		t.Fatalf("Increment returned an error: %v", err)
  1304  	}
  1305  
  1306  	if result != 1 {
  1307  		t.Fatalf("Increment's result is %d, want 1", result)
  1308  	}
  1309  
  1310  	incRequest, err = hrpc.NewIncStrSingle(context.Background(), table, key, "cf", "a", 5)
  1311  	result, err = c.Increment(incRequest)
  1312  	if err != nil {
  1313  		t.Fatalf("Increment returned an error: %v", err)
  1314  	}
  1315  
  1316  	if result != 6 {
  1317  		t.Fatalf("Increment's result is %d, want 6", result)
  1318  	}
  1319  }
  1320  
  1321  func TestIncrementParallel(t *testing.T) {
  1322  	c := gohbase.NewClient(*host)
  1323  	defer c.Close()
  1324  	key := "row102.5"
  1325  
  1326  	numParallel := 10
  1327  
  1328  	// test incerement
  1329  	var wg sync.WaitGroup
  1330  	for i := 0; i < numParallel; i++ {
  1331  		wg.Add(1)
  1332  		go func() {
  1333  			defer wg.Done()
  1334  			incRequest, err := hrpc.NewIncStrSingle(context.Background(), table, key, "cf", "a", 1)
  1335  			_, err = c.Increment(incRequest)
  1336  			if err != nil {
  1337  				t.Errorf("Increment returned an error: %v", err)
  1338  			}
  1339  		}()
  1340  	}
  1341  	wg.Wait()
  1342  
  1343  	// do one more to check if there's a correct value
  1344  	incRequest, err := hrpc.NewIncStrSingle(context.Background(), table, key, "cf", "a", 1)
  1345  	result, err := c.Increment(incRequest)
  1346  	if err != nil {
  1347  		t.Fatalf("Increment returned an error: %v", err)
  1348  	}
  1349  
  1350  	if result != int64(numParallel+1) {
  1351  		t.Fatalf("Increment's result is %d, want %d", result, numParallel+1)
  1352  	}
  1353  }
  1354  
  1355  func TestCheckAndPut(t *testing.T) {
  1356  	c := gohbase.NewClient(*host)
  1357  	defer c.Close()
  1358  
  1359  	key := "row100"
  1360  	ef := "cf"
  1361  	eq := "a"
  1362  
  1363  	var castests = []struct {
  1364  		inValues        map[string]map[string][]byte
  1365  		inExpectedValue []byte
  1366  		out             bool
  1367  	}{
  1368  		{map[string]map[string][]byte{"cf": map[string][]byte{"b": []byte("2")}},
  1369  			nil, true}, // nil instead of empty byte array
  1370  		{map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}},
  1371  			[]byte{}, true},
  1372  		{map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}},
  1373  			[]byte{}, false},
  1374  		{map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("2")}},
  1375  			[]byte("1"), true},
  1376  		{map[string]map[string][]byte{"cf": map[string][]byte{"b": []byte("2")}},
  1377  			[]byte("2"), true}, // put diff column
  1378  		{map[string]map[string][]byte{"cf": map[string][]byte{"b": []byte("2")}},
  1379  			[]byte{}, false}, // diff column
  1380  		{map[string]map[string][]byte{"cf": map[string][]byte{
  1381  			"b": []byte("100"),
  1382  			"a": []byte("100"),
  1383  		}}, []byte("2"), true}, // multiple values
  1384  	}
  1385  
  1386  	for _, tt := range castests {
  1387  		putRequest, err := hrpc.NewPutStr(context.Background(), table, key, tt.inValues)
  1388  		if err != nil {
  1389  			t.Fatalf("NewPutStr returned an error: %v", err)
  1390  		}
  1391  
  1392  		casRes, err := c.CheckAndPut(putRequest, ef, eq, tt.inExpectedValue)
  1393  
  1394  		if err != nil {
  1395  			t.Fatalf("CheckAndPut error: %s", err)
  1396  		}
  1397  
  1398  		if casRes != tt.out {
  1399  			t.Errorf("CheckAndPut with put values=%q and expectedValue=%q returned %v, want %v",
  1400  				tt.inValues, tt.inExpectedValue, casRes, tt.out)
  1401  		}
  1402  	}
  1403  
  1404  	// TODO: check the resulting state by performing a Get request
  1405  }
  1406  
  1407  func TestCheckAndPutNotPut(t *testing.T) {
  1408  	key := "row101"
  1409  	c := gohbase.NewClient(*host)
  1410  	defer c.Close()
  1411  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("lol")}}
  1412  
  1413  	appRequest, err := hrpc.NewAppStr(context.Background(), table, key, values)
  1414  	_, err = c.CheckAndPut(appRequest, "cf", "a", []byte{})
  1415  	if err == nil {
  1416  		t.Error("CheckAndPut: should not allow anything but Put request")
  1417  	}
  1418  }
  1419  
  1420  func TestCheckAndPutParallel(t *testing.T) {
  1421  	c := gohbase.NewClient(*host)
  1422  	defer c.Close()
  1423  
  1424  	keyPrefix := "row100.5"
  1425  
  1426  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}}
  1427  	capTestFunc := func(p *hrpc.Mutate, ch chan bool) {
  1428  		casRes, err := c.CheckAndPut(p, "cf", "a", []byte{})
  1429  
  1430  		if err != nil {
  1431  			t.Errorf("CheckAndPut error: %s", err)
  1432  		}
  1433  
  1434  		ch <- casRes
  1435  	}
  1436  
  1437  	// make 10 pairs of CheckAndPut requests
  1438  	for i := 0; i < 10; i++ {
  1439  		ch := make(chan bool, 2)
  1440  		putRequest1, err := hrpc.NewPutStr(
  1441  			context.Background(), table, keyPrefix+fmt.Sprint(i), values)
  1442  		if err != nil {
  1443  			t.Fatalf("NewPutStr returned an error: %v", err)
  1444  		}
  1445  		putRequest2, err := hrpc.NewPutStr(
  1446  			context.Background(), table, keyPrefix+fmt.Sprint(i), values)
  1447  		if err != nil {
  1448  			t.Fatalf("NewPutStr returned an error: %v", err)
  1449  		}
  1450  
  1451  		go capTestFunc(putRequest1, ch)
  1452  		go capTestFunc(putRequest2, ch)
  1453  
  1454  		first := <-ch
  1455  		second := <-ch
  1456  
  1457  		if first && second {
  1458  			t.Error("CheckAndPut: both requests cannot succeed")
  1459  		}
  1460  
  1461  		if !first && !second {
  1462  			t.Error("CheckAndPut: both requests cannot fail")
  1463  		}
  1464  	}
  1465  }
  1466  
  1467  func TestClose(t *testing.T) {
  1468  	c := gohbase.NewClient(*host)
  1469  
  1470  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}}
  1471  	r, err := hrpc.NewPutStr(context.Background(), table, t.Name(), values)
  1472  	if err != nil {
  1473  		t.Fatal(err)
  1474  	}
  1475  
  1476  	_, err = c.Put(r)
  1477  	if err != nil {
  1478  		t.Fatal(err)
  1479  	}
  1480  
  1481  	c.Close()
  1482  
  1483  	_, err = c.Put(r)
  1484  	if err != gohbase.ErrClientClosed {
  1485  		t.Fatalf("unexpected error: %v", err)
  1486  	}
  1487  }
  1488  
  1489  func TestCloseWithoutMeta(t *testing.T) {
  1490  	c := gohbase.NewClient(*host)
  1491  	c.Close()
  1492  
  1493  	values := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte("1")}}
  1494  	r, err := hrpc.NewPutStr(context.Background(), table, t.Name(), values)
  1495  	if err != nil {
  1496  		t.Fatal(err)
  1497  	}
  1498  
  1499  	_, err = c.Put(r)
  1500  	if err != gohbase.ErrClientClosed {
  1501  		t.Fatalf("unexpected error: %v", err)
  1502  	}
  1503  }
  1504  
  1505  // Note: This function currently causes an infinite loop in the client throwing the error -
  1506  // 2015/06/19 14:34:11 Encountered an error while reading: Failed to read from the RS: EOF
  1507  func TestChangingRegionServers(t *testing.T) {
  1508  	key := "row8"
  1509  	val := []byte("1")
  1510  	headers := map[string][]string{"cf": nil}
  1511  	if host == nil {
  1512  		t.Fatal("Host is not set!")
  1513  	}
  1514  	c := gohbase.NewClient(*host)
  1515  	defer c.Close()
  1516  	err := insertKeyValue(c, key, "cf", val)
  1517  	if err != nil {
  1518  		t.Errorf("Put returned an error: %v", err)
  1519  	}
  1520  
  1521  	// RegionServer 1 hosts all the current regions.
  1522  	// Now launch servers 2,3
  1523  	LocalRegionServersCmd(t, "start", []string{"2", "3"})
  1524  
  1525  	// Now (gracefully) stop servers 1,2.
  1526  	// All regions should now be on server 3.
  1527  	LocalRegionServersCmd(t, "stop", []string{"1", "2"})
  1528  	get, err := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
  1529  	rsp, err := c.Get(get)
  1530  	if err != nil {
  1531  		t.Errorf("Get returned an error: %v", err)
  1532  	}
  1533  	rsp_value := rsp.Cells[0].Value
  1534  	if !bytes.Equal(rsp_value, val) {
  1535  		t.Errorf("Get returned an incorrect result. Expected: %v, Received: %v",
  1536  			val, rsp_value)
  1537  	}
  1538  
  1539  	// Clean up by re-launching RS1 and closing RS3
  1540  	LocalRegionServersCmd(t, "start", []string{"1"})
  1541  	LocalRegionServersCmd(t, "stop", []string{"3"})
  1542  }
  1543  
  1544  func BenchmarkPut(b *testing.B) {
  1545  	b.ReportAllocs()
  1546  	keyPrefix := "row9"
  1547  	err := performNPuts(keyPrefix, b.N)
  1548  	if err != nil {
  1549  		b.Errorf("Put returned an error: %v", err)
  1550  	}
  1551  }
  1552  
  1553  func BenchmarkGet(b *testing.B) {
  1554  	b.ReportAllocs()
  1555  	keyPrefix := "row10"
  1556  	err := performNPuts(keyPrefix, b.N)
  1557  	if err != nil {
  1558  		b.Errorf("Put returned an error: %v", err)
  1559  	}
  1560  	c := gohbase.NewClient(*host)
  1561  	defer c.Close()
  1562  	b.ResetTimer()
  1563  	headers := map[string][]string{"cf": nil}
  1564  	for i := 0; i < b.N; i++ {
  1565  		key := keyPrefix + fmt.Sprintf("%d", i)
  1566  		get, _ := hrpc.NewGetStr(context.Background(), table, key, hrpc.Families(headers))
  1567  		c.Get(get)
  1568  	}
  1569  }
  1570  
  1571  // Helper function. Given a key_prefix, num_ops, performs num_ops.
  1572  func performNPuts(keyPrefix string, num_ops int) error {
  1573  	c := gohbase.NewClient(*host)
  1574  	defer c.Close()
  1575  	for i := 0; i < num_ops; i++ {
  1576  		key := keyPrefix + fmt.Sprintf("%d", i)
  1577  		err := insertKeyValue(c, key, "cf", []byte(fmt.Sprintf("%d", i)))
  1578  		if err != nil {
  1579  			return err
  1580  		}
  1581  	}
  1582  	return nil
  1583  }
  1584  
  1585  // Helper function. Given a client, key, columnFamily, value inserts into the table under column 'a'
  1586  func insertKeyValue(c gohbase.Client, key, columnFamily string, value []byte,
  1587  	options ...func(hrpc.Call) error) error {
  1588  	return insertKeyValueAtCol(c, key, "a", columnFamily, value, options...)
  1589  }
  1590  
  1591  // insertKeyValueAtCol inserts a value into the table under column col at key.
  1592  func insertKeyValueAtCol(c gohbase.Client, key, col, columnFamily string, value []byte,
  1593  	options ...func(hrpc.Call) error) error {
  1594  	values := map[string]map[string][]byte{columnFamily: map[string][]byte{}}
  1595  	values[columnFamily][col] = value
  1596  	putRequest, err := hrpc.NewPutStr(context.Background(), table, key, values, options...)
  1597  	if err != nil {
  1598  		return err
  1599  	}
  1600  	_, err = c.Put(putRequest)
  1601  	return err
  1602  }
  1603  
  1604  func deleteKeyValue(c gohbase.Client, key, columnFamily string, value []byte,
  1605  	options ...func(hrpc.Call) error) error {
  1606  	values := map[string]map[string][]byte{columnFamily: map[string][]byte{}}
  1607  	values[columnFamily]["a"] = value
  1608  	d, err := hrpc.NewDel(context.Background(), []byte(table), []byte(key), values)
  1609  	if err != nil {
  1610  		return err
  1611  	}
  1612  	_, err = c.Delete(d)
  1613  	return err
  1614  }
  1615  
  1616  func TestMaxResultsPerColumnFamilyGet(t *testing.T) {
  1617  	c := gohbase.NewClient(*host)
  1618  	defer c.Close()
  1619  	key := "variablecolumnrow"
  1620  	baseErr := "MaxResultsPerColumnFamilyGet error "
  1621  
  1622  	values := make(map[string]map[string][]byte)
  1623  	values["cf"] = map[string][]byte{}
  1624  
  1625  	// Save a row with 20 columns
  1626  	for i := 0; i < 20; i++ {
  1627  		colKey := fmt.Sprintf("%02d", i)
  1628  		values["cf"][colKey] = []byte(fmt.Sprintf("value %d", i))
  1629  	}
  1630  
  1631  	// First test that the function can't be used on types other than get or scan
  1632  	putRequest, err := hrpc.NewPutStr(context.Background(),
  1633  		table,
  1634  		key,
  1635  		values,
  1636  		hrpc.MaxResultsPerColumnFamily(5),
  1637  	)
  1638  	if err == nil {
  1639  		t.Errorf(baseErr+"- Option allowed to be used with incorrect type: %s", err)
  1640  	}
  1641  	putRequest, err = hrpc.NewPutStr(context.Background(),
  1642  		table,
  1643  		key,
  1644  		values,
  1645  		hrpc.ResultOffset(5),
  1646  	)
  1647  	if err == nil {
  1648  		t.Errorf(baseErr+"- Option allowed to be used with incorrect type: %s", err)
  1649  	}
  1650  
  1651  	// Now actually save the values
  1652  	putRequest, err = hrpc.NewPutStr(context.Background(), table, key, values)
  1653  	if err != nil {
  1654  		t.Errorf(baseErr+"building put string: %s", err)
  1655  
  1656  	}
  1657  	_, err = c.Put(putRequest)
  1658  	if err != nil {
  1659  		t.Errorf(baseErr+"saving row: %s", err)
  1660  	}
  1661  
  1662  	family := hrpc.Families(map[string][]string{"cf": nil})
  1663  	// Do we get the correct number of cells without qualifier
  1664  	getRequest, err := hrpc.NewGetStr(context.Background(),
  1665  		table,
  1666  		key,
  1667  		family,
  1668  		hrpc.MaxVersions(1),
  1669  	)
  1670  	if err != nil {
  1671  		t.Errorf(baseErr+"building get request: %s", err)
  1672  	}
  1673  	result, err := c.Get(getRequest)
  1674  	if len(result.Cells) != 20 {
  1675  		t.Errorf(baseErr+"- expecting %d results with parameters; received %d",
  1676  			20,
  1677  			len(result.Cells),
  1678  		)
  1679  	}
  1680  
  1681  	// Simple test for max columns per column family. Return the first n columns in order
  1682  	for testCnt := 1; testCnt <= 20; testCnt++ {
  1683  		// Get the first n columns
  1684  		getRequest, err := hrpc.NewGetStr(context.Background(),
  1685  			table,
  1686  			key,
  1687  			family,
  1688  			hrpc.MaxVersions(1),
  1689  			hrpc.MaxResultsPerColumnFamily(uint32(testCnt)),
  1690  		)
  1691  		if err != nil {
  1692  			t.Errorf(baseErr+"building get request: %s", err)
  1693  		}
  1694  		result, err := c.Get(getRequest)
  1695  		if len(result.Cells) != testCnt {
  1696  			t.Errorf(baseErr+"- expecting %d results; received %d", testCnt, len(result.Cells))
  1697  		}
  1698  		for i, x := range result.Cells {
  1699  			// Make sure the column name and value are what is expected and in correct sequence
  1700  			if string(x.Qualifier) != fmt.Sprintf("%02d", i) ||
  1701  				string(x.Value) != fmt.Sprintf("value %d", i) {
  1702  				t.Errorf(baseErr+"- unexpected return value. Expecting %s received %s",
  1703  					fmt.Sprintf("value %d", i),
  1704  					string(x.Value),
  1705  				)
  1706  			}
  1707  		}
  1708  
  1709  		// Get with out of range values
  1710  		getRequest, err = hrpc.NewGetStr(context.Background(),
  1711  			table,
  1712  			key,
  1713  			family,
  1714  			hrpc.MaxVersions(1),
  1715  			hrpc.MaxResultsPerColumnFamily(math.MaxUint32),
  1716  		)
  1717  		if err == nil {
  1718  			t.Error(baseErr + "- out of range column result parameter accepted")
  1719  		}
  1720  		// Get with out of range values
  1721  		getRequest, err = hrpc.NewGetStr(context.Background(),
  1722  			table,
  1723  			key,
  1724  			family,
  1725  			hrpc.MaxVersions(1),
  1726  			hrpc.ResultOffset(math.MaxUint32),
  1727  		)
  1728  		if err == nil {
  1729  			t.Error(baseErr + "- out of range column offset parameter accepted")
  1730  		}
  1731  
  1732  	}
  1733  
  1734  	// Max columns per column family. Return first n cells in order with offset.
  1735  	for offset := 0; offset < 20; offset++ {
  1736  		for maxResults := 1; maxResults <= 20-offset; maxResults++ {
  1737  			getRequest, err := hrpc.NewGetStr(context.Background(),
  1738  				table,
  1739  				key,
  1740  				family,
  1741  				hrpc.MaxVersions(1),
  1742  				hrpc.MaxResultsPerColumnFamily(uint32(maxResults)),
  1743  				hrpc.ResultOffset(uint32(offset)),
  1744  			)
  1745  			if err != nil {
  1746  				t.Errorf(baseErr+"building get request testing offset: %s", err)
  1747  			}
  1748  			result, err := c.Get(getRequest)
  1749  
  1750  			// Make sure number of cells returned is still correct
  1751  			if len(result.Cells) != maxResults {
  1752  				t.Errorf(baseErr+"with offset - expecting %d results; received %d",
  1753  					maxResults,
  1754  					len(result.Cells),
  1755  				)
  1756  			}
  1757  			// make sure the cells returned are what is expected and in correct sequence
  1758  			for i, _ := range result.Cells {
  1759  				if string(result.Cells[i].Value) != fmt.Sprintf("value %d", offset+i) {
  1760  					t.Errorf(baseErr+"with offset - Expected value %s but received %s",
  1761  						fmt.Sprintf("value %d", offset+i),
  1762  						string(result.Cells[i].Value),
  1763  					)
  1764  				}
  1765  			}
  1766  		}
  1767  	}
  1768  }
  1769  
  1770  func TestMaxResultsPerColumnFamilyScan(t *testing.T) {
  1771  	c := gohbase.NewClient(*host)
  1772  	defer c.Close()
  1773  
  1774  	baseErr := "MaxResultsPerColumnFamilyScan error "
  1775  	key := "variablecolumnrow_1"
  1776  
  1777  	values := make(map[string]map[string][]byte)
  1778  	values["cf"] = map[string][]byte{}
  1779  
  1780  	// Save a row with 20 columns
  1781  	for i := 0; i < 20; i++ {
  1782  		colKey := fmt.Sprintf("%02d", i)
  1783  		values["cf"][colKey] = []byte(fmt.Sprintf("value %d", i))
  1784  	}
  1785  	putRequest, err := hrpc.NewPutStr(context.Background(), table, key, values)
  1786  	if err != nil {
  1787  		t.Errorf(baseErr+"building put string: %s", err)
  1788  
  1789  	}
  1790  	_, err = c.Put(putRequest)
  1791  	if err != nil {
  1792  		t.Errorf(baseErr+"saving row: %s", err)
  1793  	}
  1794  	// Save another row with 20 columns
  1795  	key = "variablecolumnrow_2"
  1796  	putRequest, err = hrpc.NewPutStr(context.Background(), table, key, values)
  1797  	if err != nil {
  1798  		t.Errorf(baseErr+"building put string: %s", err)
  1799  
  1800  	}
  1801  	_, err = c.Put(putRequest)
  1802  	if err != nil {
  1803  		t.Errorf(baseErr+"saving row: %s", err)
  1804  	}
  1805  
  1806  	family := hrpc.Families(map[string][]string{"cf": nil})
  1807  	pFilter := filter.NewPrefixFilter([]byte("variablecolumnrow_"))
  1808  
  1809  	// Do we get the correct number of cells without qualifier
  1810  	scanRequest, err := hrpc.NewScanStr(context.Background(),
  1811  		table,
  1812  		family,
  1813  		hrpc.Filters(pFilter),
  1814  		hrpc.MaxVersions(1),
  1815  	)
  1816  	if err != nil {
  1817  		t.Errorf(baseErr+"building scan request: %s", err)
  1818  	}
  1819  
  1820  	result := c.Scan(scanRequest)
  1821  	resultCnt := 0
  1822  	for {
  1823  		rRow, err := result.Next()
  1824  		if err == io.EOF {
  1825  			break
  1826  		}
  1827  		if err != nil {
  1828  			t.Errorf(baseErr+"scanning result: %s", err)
  1829  		}
  1830  		if len(rRow.Cells) != 20 {
  1831  			t.Errorf(baseErr+"- expected all 20 columns but received %d", len(rRow.Cells))
  1832  		}
  1833  		resultCnt++
  1834  	}
  1835  	if resultCnt != 2 {
  1836  		t.Errorf(baseErr+"- expected 2 rows; received %d", resultCnt)
  1837  	}
  1838  
  1839  	// Do we get a limited number of columns per row
  1840  	baseErr = "MaxResultsPerColumnFamilyScan with limited columns error "
  1841  	scanRequest, err = hrpc.NewScanStr(context.Background(),
  1842  		table,
  1843  		family,
  1844  		hrpc.Filters(pFilter),
  1845  		hrpc.MaxVersions(1),
  1846  		hrpc.MaxResultsPerColumnFamily(15),
  1847  	)
  1848  	if err != nil {
  1849  		t.Errorf(baseErr+"building scan request: %s", err)
  1850  	}
  1851  
  1852  	result = c.Scan(scanRequest)
  1853  	resultCnt = 0
  1854  	for {
  1855  		rRow, err := result.Next()
  1856  		if err == io.EOF {
  1857  			break
  1858  		}
  1859  		if err != nil {
  1860  			t.Errorf(baseErr+"scanning result: %s", err)
  1861  		}
  1862  		if len(rRow.Cells) != 15 {
  1863  			t.Errorf(baseErr+"- expected 15 columns but received %d", len(rRow.Cells))
  1864  		}
  1865  		resultCnt++
  1866  	}
  1867  	if resultCnt != 2 {
  1868  		t.Errorf(baseErr+"- expected 2 rows; received %d", resultCnt)
  1869  	}
  1870  
  1871  	// Do we get a limited number of columns per row and are they correctly offset
  1872  	baseErr = "MaxResultsPerColumnFamilyScan with limited columns and offset error "
  1873  	scanRequest, err = hrpc.NewScanStr(context.Background(),
  1874  		table,
  1875  		family,
  1876  		hrpc.Filters(pFilter),
  1877  		hrpc.MaxVersions(1),
  1878  		hrpc.MaxResultSize(1),
  1879  		hrpc.MaxResultsPerColumnFamily(2),
  1880  		hrpc.ResultOffset(10),
  1881  	)
  1882  	if err != nil {
  1883  		t.Errorf(baseErr+"building scan request: %s", err)
  1884  	}
  1885  
  1886  	result = c.Scan(scanRequest)
  1887  	resultCnt = 0
  1888  	for {
  1889  		rRow, err := result.Next()
  1890  		if err == io.EOF {
  1891  			break
  1892  		}
  1893  		if err != nil {
  1894  			t.Errorf(baseErr+"scanning result: %s", err)
  1895  		}
  1896  		if len(rRow.Cells) != 2 {
  1897  			t.Errorf(baseErr+"- expected 2 columns but received %d", len(rRow.Cells))
  1898  		}
  1899  		if string(rRow.Cells[0].Value) != "value 10" || string(rRow.Cells[1].Value) != "value 11" {
  1900  			t.Errorf(baseErr+"- unexpected cells values. "+
  1901  				"Expected 'value 10' and 'value 11' - received %s and %s",
  1902  				string(rRow.Cells[0].Value),
  1903  				string(rRow.Cells[1].Value),
  1904  			)
  1905  		}
  1906  		resultCnt++
  1907  	}
  1908  	if resultCnt != 2 {
  1909  		t.Errorf(baseErr+"- expected 1 row; received %d", resultCnt)
  1910  	}
  1911  
  1912  	// Test with out of range values
  1913  	scanRequest, err = hrpc.NewScanStr(context.Background(),
  1914  		table,
  1915  		family,
  1916  		hrpc.Filters(pFilter),
  1917  		hrpc.MaxVersions(1),
  1918  		hrpc.MaxResultsPerColumnFamily(math.MaxUint32),
  1919  	)
  1920  	if err == nil {
  1921  		t.Error(baseErr + "- out of range column result parameter accepted")
  1922  	}
  1923  	scanRequest, err = hrpc.NewScanStr(context.Background(),
  1924  		table,
  1925  		family,
  1926  		hrpc.Filters(pFilter),
  1927  		hrpc.MaxVersions(1),
  1928  		hrpc.ResultOffset(math.MaxUint32),
  1929  	)
  1930  	if err == nil {
  1931  		t.Error(baseErr + "- out of range column result parameter accepted")
  1932  	}
  1933  
  1934  }
  1935  
  1936  func TestMultiRequest(t *testing.T) {
  1937  	// pre-populate the table
  1938  	var (
  1939  		getKey       = t.Name() + "_Get"
  1940  		putKey       = t.Name() + "_Put"
  1941  		deleteKey    = t.Name() + "_Delete"
  1942  		appendKey    = t.Name() + "_Append"
  1943  		incrementKey = t.Name() + "_Increment"
  1944  	)
  1945  	c := gohbase.NewClient(*host, gohbase.RpcQueueSize(1))
  1946  	if err := insertKeyValue(c, getKey, "cf", []byte{1}); err != nil {
  1947  		t.Fatal(err)
  1948  	}
  1949  	if err := insertKeyValue(c, deleteKey, "cf", []byte{3}); err != nil {
  1950  		t.Fatal(err)
  1951  	}
  1952  	if err := insertKeyValue(c, appendKey, "cf", []byte{4}); err != nil {
  1953  		t.Fatal(err)
  1954  	}
  1955  	i, err := hrpc.NewIncStrSingle(context.Background(), table, incrementKey, "cf", "a", 5)
  1956  	if err != nil {
  1957  		t.Fatal(err)
  1958  	}
  1959  	_, err = c.Increment(i)
  1960  	if err != nil {
  1961  		t.Fatal(err)
  1962  	}
  1963  	c.Close()
  1964  
  1965  	c = gohbase.NewClient(*host, gohbase.FlushInterval(1000*time.Hour), gohbase.RpcQueueSize(5))
  1966  	defer c.Close()
  1967  
  1968  	var wg sync.WaitGroup
  1969  	wg.Add(5)
  1970  	go func() {
  1971  		g, err := hrpc.NewGetStr(context.Background(), table, getKey)
  1972  		if err != nil {
  1973  			t.Error(err)
  1974  		}
  1975  		r, err := c.Get(g)
  1976  		if err != nil {
  1977  			t.Error(err)
  1978  		}
  1979  		expV := []byte{1}
  1980  		if !bytes.Equal(r.Cells[0].Value, expV) {
  1981  			t.Errorf("expected %v, got %v:", expV, r.Cells[0].Value)
  1982  		}
  1983  		wg.Done()
  1984  	}()
  1985  
  1986  	go func() {
  1987  		v := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte{2}}}
  1988  		p, err := hrpc.NewPutStr(context.Background(), table, putKey, v)
  1989  		if err != nil {
  1990  			t.Error(err)
  1991  		}
  1992  		r, err := c.Put(p)
  1993  		if err != nil {
  1994  			t.Error(err)
  1995  		}
  1996  		if len(r.Cells) != 0 {
  1997  			t.Errorf("expected no cells, got %d", len(r.Cells))
  1998  		}
  1999  		wg.Done()
  2000  	}()
  2001  
  2002  	go func() {
  2003  		d, err := hrpc.NewDelStr(context.Background(), table, deleteKey, nil)
  2004  		if err != nil {
  2005  			t.Error(err)
  2006  		}
  2007  		r, err := c.Delete(d)
  2008  		if err != nil {
  2009  			t.Error(err)
  2010  		}
  2011  		if len(r.Cells) != 0 {
  2012  			t.Errorf("expected no cells, got %d", len(r.Cells))
  2013  		}
  2014  		wg.Done()
  2015  	}()
  2016  
  2017  	go func() {
  2018  		v := map[string]map[string][]byte{"cf": map[string][]byte{"a": []byte{4}}}
  2019  		a, err := hrpc.NewAppStr(context.Background(), table, appendKey, v)
  2020  		if err != nil {
  2021  			t.Error(err)
  2022  		}
  2023  		r, err := c.Append(a)
  2024  		if err != nil {
  2025  			t.Error(err)
  2026  		}
  2027  		expV := []byte{4, 4}
  2028  		if !bytes.Equal(r.Cells[0].Value, expV) {
  2029  			t.Errorf("expected %v, got %v:", expV, r.Cells[0].Value)
  2030  		}
  2031  		wg.Done()
  2032  	}()
  2033  
  2034  	go func() {
  2035  		i, err := hrpc.NewIncStrSingle(context.Background(), table, incrementKey, "cf", "a", 1)
  2036  		if err != nil {
  2037  			t.Error(err)
  2038  		}
  2039  		r, err := c.Increment(i)
  2040  		if err != nil {
  2041  			t.Error(err)
  2042  		}
  2043  		if r != 6 {
  2044  			t.Errorf("expected %d, got %d:", 6, r)
  2045  		}
  2046  		wg.Done()
  2047  	}()
  2048  
  2049  	wg.Wait()
  2050  }
  2051  
  2052  func TestReverseScan(t *testing.T) {
  2053  	c := gohbase.NewClient(*host)
  2054  	defer c.Close()
  2055  
  2056  	baseErr := "Reverse Scan error "
  2057  
  2058  	values := make(map[string]map[string][]byte)
  2059  	values["cf"] = map[string][]byte{}
  2060  
  2061  	// Save 500 rows
  2062  	for i := 0; i < 500; i++ {
  2063  		key := fmt.Sprintf("REVTEST-%03d", i)
  2064  		values["cf"]["reversetest"] = []byte(fmt.Sprintf("%d", i))
  2065  		putRequest, err := hrpc.NewPutStr(context.Background(), table, key, values)
  2066  		if err != nil {
  2067  			t.Errorf(baseErr+"building put string: %s", err)
  2068  
  2069  		}
  2070  
  2071  		_, err = c.Put(putRequest)
  2072  		if err != nil {
  2073  			t.Errorf(baseErr+"saving row: %s", err)
  2074  		}
  2075  	}
  2076  
  2077  	// Read them back in reverse order
  2078  	scanRequest, err := hrpc.NewScanRangeStr(context.Background(),
  2079  		table,
  2080  		"REVTEST-999",
  2081  		"REVTEST-",
  2082  		hrpc.Families(map[string][]string{"cf": []string{"reversetest"}}),
  2083  		hrpc.Reversed(),
  2084  	)
  2085  	if err != nil {
  2086  		t.Errorf(baseErr+"setting up reverse scan: %s", err)
  2087  	}
  2088  	i := 0
  2089  	results := c.Scan(scanRequest)
  2090  	for {
  2091  		r, err := results.Next()
  2092  		if err != nil {
  2093  			if err == io.EOF {
  2094  				break
  2095  			}
  2096  			t.Errorf(baseErr+"scanning results: %s", err)
  2097  		}
  2098  		i++
  2099  		expected := fmt.Sprintf("%d", 500-i)
  2100  		if string(r.Cells[0].Value) != expected {
  2101  			t.Errorf(baseErr + "- unexpected rowkey returned")
  2102  		}
  2103  	}
  2104  	if i != 500 {
  2105  		t.Errorf(baseErr+" expected 500 rows returned; found %d", i)
  2106  	}
  2107  	results.Close()
  2108  
  2109  	// Read part of them back in reverse order. Stoprow should be exclusive just like forward scan
  2110  	scanRequest, err = hrpc.NewScanRangeStr(context.Background(),
  2111  		table,
  2112  		"REVTEST-250",
  2113  		"REVTEST-150",
  2114  		hrpc.Families(map[string][]string{"cf": []string{"reversetest"}}),
  2115  		hrpc.Reversed(),
  2116  	)
  2117  	if err != nil {
  2118  		t.Errorf(baseErr+"setting up reverse scan: %s", err)
  2119  	}
  2120  	i = 0
  2121  	results = c.Scan(scanRequest)
  2122  	for {
  2123  		r, err := results.Next()
  2124  		if err != nil {
  2125  			if err == io.EOF {
  2126  				break
  2127  			}
  2128  			t.Errorf(baseErr+"scanning results: %s", err)
  2129  		}
  2130  		i++
  2131  		expected := fmt.Sprintf("%d", 251-i)
  2132  		if string(r.Cells[0].Value) != expected {
  2133  			t.Errorf(baseErr + "- unexpected rowkey returned when doing partial reverse scan")
  2134  		}
  2135  	}
  2136  	if i != 100 {
  2137  		t.Errorf(baseErr+" expected 100 rows returned; found %d", i)
  2138  	}
  2139  	results.Close()
  2140  
  2141  }
  2142  
  2143  func TestListTableNames(t *testing.T) {
  2144  	// Initialize our tables
  2145  	ac := gohbase.NewAdminClient(*host)
  2146  	tables := []string{
  2147  		table + "_MATCH1",
  2148  		table + "_MATCH2",
  2149  		table + "nomatch",
  2150  	}
  2151  
  2152  	for _, tn := range tables {
  2153  		// Since this test is called by TestMain which waits for hbase init
  2154  		// there is no need to wait here.
  2155  		err := CreateTable(ac, tn, []string{"cf"})
  2156  		if err != nil {
  2157  			panic(err)
  2158  		}
  2159  	}
  2160  
  2161  	defer func() {
  2162  		for _, tn := range tables {
  2163  			err := DeleteTable(ac, tn)
  2164  			if err != nil {
  2165  				panic(err)
  2166  			}
  2167  		}
  2168  	}()
  2169  
  2170  	m1 := []byte(table + "_MATCH1")
  2171  	m2 := []byte(table + "_MATCH2")
  2172  	tcases := []struct {
  2173  		desc      string
  2174  		regex     string
  2175  		namespace string
  2176  		sys       bool
  2177  
  2178  		match []*pb.TableName
  2179  	}{
  2180  		{
  2181  			desc:  "match all",
  2182  			regex: ".*",
  2183  			match: []*pb.TableName{
  2184  				&pb.TableName{Qualifier: []byte(table)},
  2185  				&pb.TableName{Qualifier: m1},
  2186  				&pb.TableName{Qualifier: m2},
  2187  				&pb.TableName{Qualifier: []byte(table + "nomatch")},
  2188  			},
  2189  		},
  2190  		{
  2191  			desc:  "match_some",
  2192  			regex: ".*_MATCH.*",
  2193  			match: []*pb.TableName{
  2194  				&pb.TableName{Qualifier: m1},
  2195  				&pb.TableName{Qualifier: m2},
  2196  			},
  2197  		},
  2198  		{
  2199  			desc: "match_none",
  2200  		},
  2201  		{
  2202  			desc:      "match meta",
  2203  			regex:     ".*meta.*",
  2204  			namespace: "hbase",
  2205  			sys:       true,
  2206  			match: []*pb.TableName{
  2207  				&pb.TableName{Qualifier: []byte("meta")},
  2208  			},
  2209  		},
  2210  	}
  2211  
  2212  	for _, tcase := range tcases {
  2213  		t.Run(tcase.desc, func(t *testing.T) {
  2214  			tn, err := hrpc.NewListTableNames(
  2215  				context.Background(),
  2216  				hrpc.ListRegex(tcase.regex),
  2217  				hrpc.ListSysTables(tcase.sys),
  2218  				hrpc.ListNamespace(tcase.namespace),
  2219  			)
  2220  			if err != nil {
  2221  				t.Fatal(err)
  2222  			}
  2223  
  2224  			names, err := ac.ListTableNames(tn)
  2225  			if err != nil {
  2226  				t.Error(err)
  2227  			}
  2228  
  2229  			// filter to have only tables that we've created
  2230  			var got []*pb.TableName
  2231  			for _, m := range names {
  2232  				if strings.HasPrefix(string(m.Qualifier), table) ||
  2233  					string(m.Namespace) == "hbase" {
  2234  					got = append(got, m)
  2235  				}
  2236  			}
  2237  
  2238  			if len(got) != len(tcase.match) {
  2239  				t.Errorf("expected %v, got %v", tcase.match, got)
  2240  			}
  2241  
  2242  			for i, m := range tcase.match {
  2243  				want := string(m.Qualifier)
  2244  				got := string(tcase.match[i].Qualifier)
  2245  				if want != got {
  2246  					t.Errorf("index %d: expected: %v, got %v", i, want, got)
  2247  				}
  2248  			}
  2249  		})
  2250  	}
  2251  
  2252  }
  2253  
  2254  // Test snapshot creation
  2255  func TestSnapshot(t *testing.T) {
  2256  	ac := gohbase.NewAdminClient(*host)
  2257  
  2258  	name := "snapshot-" + table
  2259  
  2260  	sn, err := hrpc.NewSnapshot(context.Background(), name, table)
  2261  	if err != nil {
  2262  		t.Fatal(err)
  2263  	}
  2264  	if err = ac.CreateSnapshot(sn); err != nil {
  2265  		t.Error(err)
  2266  	}
  2267  
  2268  	defer func() {
  2269  		if err = ac.DeleteSnapshot(sn); err != nil {
  2270  			t.Error(err)
  2271  		}
  2272  	}()
  2273  
  2274  	ls := hrpc.NewListSnapshots(context.Background())
  2275  	snaps, err := ac.ListSnapshots(ls)
  2276  	if err != nil {
  2277  		t.Error(err)
  2278  	}
  2279  
  2280  	if len(snaps) != 1 {
  2281  		t.Errorf("expection 1 snapshot, got %v", len(snaps))
  2282  	}
  2283  
  2284  	gotName := snaps[0].GetName()
  2285  	if gotName != name {
  2286  		t.Errorf("expection snapshot name to be %v got %v", name, gotName)
  2287  	}
  2288  }
  2289  
  2290  // TestRestoreSnapshot tests using a snapshot to restore a table.
  2291  func TestRestoreSnapshot(t *testing.T) {
  2292  	// Procedure for this test is roughly:
  2293  	// - Create some data in a table.
  2294  	// - Create a snapshot.
  2295  	// - Remove all data.
  2296  	// - Restore snapshot.
  2297  	// - Ensure data is there.
  2298  
  2299  	var (
  2300  		key  = t.Name() + "_Get"
  2301  		name = "snapshot-" + table
  2302  	)
  2303  
  2304  	c := gohbase.NewClient(*host, gohbase.RpcQueueSize(1))
  2305  	if err := insertKeyValue(c, key, "cf", []byte{1}); err != nil {
  2306  		t.Fatal(err)
  2307  	}
  2308  
  2309  	ac := gohbase.NewAdminClient(*host)
  2310  
  2311  	sn, err := hrpc.NewSnapshot(context.Background(), name, table)
  2312  	if err != nil {
  2313  		t.Fatal(err)
  2314  	}
  2315  	if err := ac.CreateSnapshot(sn); err != nil {
  2316  		t.Error(err)
  2317  	}
  2318  
  2319  	defer func() {
  2320  		err = ac.DeleteSnapshot(sn)
  2321  		if err != nil {
  2322  			t.Error(err)
  2323  		}
  2324  	}()
  2325  
  2326  	if err := deleteKeyValue(c, key, "cf", []byte{1}); err != nil {
  2327  		t.Error(err)
  2328  	}
  2329  
  2330  	g, err := hrpc.NewGetStr(context.Background(), table, key)
  2331  	if err != nil {
  2332  		t.Error(err)
  2333  	}
  2334  
  2335  	r, err := c.Get(g)
  2336  	if err != nil {
  2337  		t.Error(err)
  2338  	}
  2339  
  2340  	if len(r.Cells) != 0 {
  2341  		t.Fatalf("expected no cells in table %s key %s", table, key)
  2342  	}
  2343  
  2344  	c.Close()
  2345  
  2346  	td := hrpc.NewDisableTable(context.Background(), []byte(table))
  2347  	if err := ac.DisableTable(td); err != nil {
  2348  		t.Error(err)
  2349  	}
  2350  
  2351  	if err = ac.RestoreSnapshot(sn); err != nil {
  2352  		t.Error(err)
  2353  	}
  2354  
  2355  	te := hrpc.NewEnableTable(context.Background(), []byte(table))
  2356  	if err := ac.EnableTable(te); err != nil {
  2357  		t.Error(err)
  2358  	}
  2359  
  2360  	c = gohbase.NewClient(*host, gohbase.RpcQueueSize(1))
  2361  
  2362  	r, err = c.Get(g)
  2363  	if err != nil {
  2364  		t.Error(err)
  2365  	}
  2366  
  2367  	expV := []byte{1}
  2368  	if !bytes.Equal(r.Cells[0].Value, expV) {
  2369  		t.Errorf("expected %v, got %v:", expV, r.Cells[0].Value)
  2370  	}
  2371  }
  2372  
  2373  func TestSetBalancer(t *testing.T) {
  2374  	ac := gohbase.NewAdminClient(*host)
  2375  
  2376  	sb, err := hrpc.NewSetBalancer(context.Background(), false)
  2377  	if err != nil {
  2378  		t.Fatal(err)
  2379  	}
  2380  	prevState, err := ac.SetBalancer(sb)
  2381  	if err != nil {
  2382  		t.Fatal(err)
  2383  	}
  2384  	if !prevState {
  2385  		t.Fatal("expected balancer to be previously enabled")
  2386  	}
  2387  
  2388  	sb, err = hrpc.NewSetBalancer(context.Background(), true)
  2389  	if err != nil {
  2390  		t.Fatal(err)
  2391  	}
  2392  	prevState, err = ac.SetBalancer(sb)
  2393  	if err != nil {
  2394  		t.Fatal(err)
  2395  	}
  2396  	if prevState {
  2397  		t.Fatal("expected balancer to be previously disabled")
  2398  	}
  2399  }
  2400  
  2401  func TestMoveRegion(t *testing.T) {
  2402  	c := gohbase.NewClient(*host)
  2403  	ac := gohbase.NewAdminClient(*host)
  2404  
  2405  	// scan meta to get a region to move
  2406  	scan, err := hrpc.NewScan(context.Background(),
  2407  		[]byte("hbase:meta"),
  2408  		hrpc.Families(map[string][]string{"info": []string{"regioninfo"}}))
  2409  	if err != nil {
  2410  		t.Fatal(err)
  2411  	}
  2412  
  2413  	var rsp []*hrpc.Result
  2414  	scanner := c.Scan(scan)
  2415  	for {
  2416  		res, err := scanner.Next()
  2417  		if err == io.EOF {
  2418  			break
  2419  		}
  2420  		if err != nil {
  2421  			t.Fatal(err)
  2422  		}
  2423  		rsp = append(rsp, res)
  2424  	}
  2425  
  2426  	// use the first region
  2427  	if len(rsp) == 0 {
  2428  		t.Fatal("got 0 results")
  2429  	}
  2430  	if len(rsp[0].Cells) == 0 {
  2431  		t.Fatal("got 0 cells")
  2432  	}
  2433  
  2434  	regionName := rsp[0].Cells[0].Row
  2435  	regionName = regionName[len(regionName)-33 : len(regionName)-1]
  2436  	mr, err := hrpc.NewMoveRegion(context.Background(), regionName)
  2437  	if err != nil {
  2438  		t.Fatal(err)
  2439  	}
  2440  
  2441  	if err := ac.MoveRegion(mr); err != nil {
  2442  		t.Fatal(err)
  2443  	}
  2444  }
  2445  
  2446  func TestDebugState(t *testing.T) {
  2447  	key := "row1"
  2448  	val := []byte("1")
  2449  	if host == nil {
  2450  		t.Fatal("Host is not set!")
  2451  	}
  2452  
  2453  	c := gohbase.NewClient(*host)
  2454  	defer c.Close()
  2455  	err := insertKeyValue(c, key, "cf", val)
  2456  	if err != nil {
  2457  		t.Fatalf("Put returned an error: %v", err)
  2458  	}
  2459  
  2460  	jsonVal, err := gohbase.DebugState(c)
  2461  
  2462  	if err != nil {
  2463  		t.Fatalf("DebugState returned an error when it shouldn't have: %v", err)
  2464  	}
  2465  
  2466  	var jsonUnMarshal map[string]interface{}
  2467  	err = json.Unmarshal(jsonVal, &jsonUnMarshal)
  2468  
  2469  	if err != nil {
  2470  		t.Fatalf("Encoutered eror when Unmarshalling: %v", err)
  2471  	}
  2472  
  2473  	clientRegionMap := jsonUnMarshal["ClientRegionMap"]
  2474  	clientType := jsonUnMarshal["ClientType"]
  2475  	regionInfoMap := jsonUnMarshal["RegionInfoMap"]
  2476  	keyRegionCache := jsonUnMarshal["KeyRegionCache"]
  2477  	clientRegionCache := jsonUnMarshal["ClientRegionCache"]
  2478  
  2479  	expectedClientRegionSize := 1
  2480  	regionInfoMapSize := 2
  2481  
  2482  	assert.Equal(t, string(region.RegionClient), clientType.(string))
  2483  	assert.Equal(t, expectedClientRegionSize, len(clientRegionMap.(map[string]interface{})))
  2484  	assert.Equal(t, regionInfoMapSize, len(regionInfoMap.(map[string]interface{})))
  2485  	assert.Equal(t, 1, len(keyRegionCache.(map[string]interface{})))
  2486  	assert.Equal(t, 1, len(clientRegionCache.(map[string]interface{}))) // only have one client
  2487  }
  2488  
  2489  type regionInfoAndAddr struct {
  2490  	regionInfo hrpc.RegionInfo
  2491  	addr       string
  2492  }
  2493  
  2494  // Test loading region cache
  2495  func TestCacheRegions(t *testing.T) {
  2496  	c := gohbase.NewClient(*host)
  2497  	defer c.Close()
  2498  
  2499  	// make sure region cache is empty at startup
  2500  	var jsonUnMarshalStart map[string]interface{}
  2501  	jsonVal, err := gohbase.DebugState(c)
  2502  	err = json.Unmarshal(jsonVal, &jsonUnMarshalStart)
  2503  	if err != nil {
  2504  		t.Fatalf("Encoutered eror when Unmarshalling: %v", err)
  2505  	}
  2506  	cacheLength := len(jsonUnMarshalStart["KeyRegionCache"].(map[string]interface{}))
  2507  	if cacheLength != 0 {
  2508  		t.Fatal("expected empty region cache when creating a new client")
  2509  	}
  2510  
  2511  	c.CacheRegions([]byte(table))
  2512  
  2513  	var jsonUnMarshalCached map[string]interface{}
  2514  	jsonVal, err = gohbase.DebugState(c)
  2515  	err = json.Unmarshal(jsonVal, &jsonUnMarshalCached)
  2516  	if err != nil {
  2517  		t.Fatalf("Encoutered eror when Unmarshalling: %v", err)
  2518  	}
  2519  	// CreateTable init function starts hbase with 4 regions
  2520  	cacheLength = len(jsonUnMarshalCached["KeyRegionCache"].(map[string]interface{}))
  2521  	if cacheLength != 4 {
  2522  		t.Fatalf("Expect 4 regions but got: %v", cacheLength)
  2523  	}
  2524  
  2525  }
  2526  
  2527  // TestNewTableFromSnapshot tests the ability to create a snapshot from a table,
  2528  // and then use this snapshot to create a new, different table from the table the
  2529  // snapshot was created from. This is different from restoring the snapshot to the
  2530  // table it was created from.
  2531  func TestNewTableFromSnapshot(t *testing.T) {
  2532  	var (
  2533  		key          = t.Name() + "_Get"
  2534  		snapshotName = "snapshot-" + table
  2535  	)
  2536  
  2537  	c := gohbase.NewClient(*host, gohbase.RpcQueueSize(1))
  2538  	defer c.Close()
  2539  	// Insert some data into the main test table.
  2540  	if err := insertKeyValue(c, key, "cf", []byte{1}); err != nil {
  2541  		t.Fatal(err)
  2542  	}
  2543  
  2544  	ac := gohbase.NewAdminClient(*host)
  2545  	// Create snapshot from the main test table.
  2546  	sn, err := hrpc.NewSnapshot(context.Background(), snapshotName, table)
  2547  	if err != nil {
  2548  		t.Fatal(err)
  2549  	}
  2550  	if err = ac.CreateSnapshot(sn); err != nil {
  2551  		t.Fatal(err)
  2552  	}
  2553  
  2554  	defer func() {
  2555  		err = ac.DeleteSnapshot(sn)
  2556  		if err != nil {
  2557  			t.Error(err)
  2558  		}
  2559  	}()
  2560  
  2561  	// Delete the data from main test table after taking snapshot.
  2562  	if err = deleteKeyValue(c, key, "cf", []byte{1}); err != nil {
  2563  		t.Fatal(err)
  2564  	}
  2565  	// Confirm data has been deleted.
  2566  	gMain, err := hrpc.NewGetStr(context.Background(), table, key)
  2567  	if err != nil {
  2568  		t.Fatal(err)
  2569  	}
  2570  
  2571  	r, err := c.Get(gMain)
  2572  	if err != nil {
  2573  		t.Fatal(err)
  2574  	}
  2575  
  2576  	if len(r.Cells) != 0 {
  2577  		t.Fatalf("expected no cells in table %s key %s", table, key)
  2578  	}
  2579  
  2580  	// Restore the snapshot of the same name to a new table.
  2581  	// The new table doesn't exist yet, HBase will create it when trying to restore a snapshot to a
  2582  	// table that does not already exist. If the snapshot table doesn't exist, as in this case,
  2583  	// HBase will clone the snapshot to a new table.
  2584  	tableNew := fmt.Sprintf("gohbase_test_%d_%s", time.Now().UnixNano(), t.Name())
  2585  	sn, err = hrpc.NewSnapshot(context.Background(), snapshotName, tableNew)
  2586  	if err != nil {
  2587  		t.Fatal(err)
  2588  	}
  2589  	if err = ac.RestoreSnapshot(sn); err != nil {
  2590  		t.Fatal(err)
  2591  	}
  2592  
  2593  	// It may take some time for the new table with the restored data to be created,
  2594  	// wait some time for this to complete.
  2595  	var tn *hrpc.ListTableNames
  2596  	ctx, _ := context.WithTimeout(context.Background(), 30*time.Second)
  2597  	tn, err = hrpc.NewListTableNames(ctx, hrpc.ListRegex(tableNew))
  2598  	for {
  2599  		var names []*pb.TableName
  2600  		names, err = ac.ListTableNames(tn)
  2601  		if err != nil {
  2602  			t.Fatal(err)
  2603  		}
  2604  		if len(names) != 0 {
  2605  			break
  2606  		}
  2607  		time.Sleep(1 * time.Second)
  2608  	}
  2609  
  2610  	// Now that this table has been created, clean up after test.
  2611  	defer func() {
  2612  		err = DeleteTable(ac, tableNew)
  2613  		if err != nil {
  2614  			t.Fatal(err)
  2615  		}
  2616  	}()
  2617  
  2618  	// Check that the snapshot data has been cloned to the new table.
  2619  	gNew, err := hrpc.NewGetStr(context.Background(), tableNew, key)
  2620  	if err != nil {
  2621  		t.Fatal(err)
  2622  	}
  2623  	r, err = c.Get(gNew)
  2624  	if err != nil {
  2625  		t.Fatal(err)
  2626  	}
  2627  	if len(r.Cells) == 0 {
  2628  		t.Fatal("Expected non-empty result")
  2629  	}
  2630  	expV := []byte{1}
  2631  	if !bytes.Equal(r.Cells[0].Value, expV) {
  2632  		t.Fatalf("expected %v, got %v:", expV, r.Cells[0].Value)
  2633  	}
  2634  
  2635  	// Checking that the data did not get restored to the main test table:
  2636  	r, err = c.Get(gMain)
  2637  	if err != nil {
  2638  		t.Fatal(err)
  2639  	}
  2640  
  2641  	if len(r.Cells) != 0 {
  2642  		t.Fatalf("expected no cells after RestoreSnapshot in table %s key %s", table, key)
  2643  	}
  2644  }
  2645  
  2646  // TestScannerTimeout makes sure that without the Renew flag on we get
  2647  // a lease timeout between Next calls if the wait between them is too long.
  2648  func TestScannerTimeout(t *testing.T) {
  2649  	c := gohbase.NewClient(*host)
  2650  	defer c.Close()
  2651  	// Insert test data
  2652  	keyPrefix := "scanner_timeout_test_"
  2653  	numRows := 2
  2654  	for i := 0; i < numRows; i++ {
  2655  		key := fmt.Sprintf("%s%d", keyPrefix, i)
  2656  		value := []byte(strconv.Itoa(i))
  2657  		err := insertKeyValue(c, key, "cf", value)
  2658  		if err != nil {
  2659  			t.Fatalf("Failed to insert test data: %v", err)
  2660  		}
  2661  	}
  2662  
  2663  	// Create a scan request
  2664  	// renewal is default set to false
  2665  	// We set result size to 1 to force a lease timeout between Next calls
  2666  	scan, err := hrpc.NewScanStr(context.Background(), table,
  2667  		hrpc.Families(map[string][]string{"cf": nil}),
  2668  		hrpc.Filters(filter.NewPrefixFilter([]byte(keyPrefix))),
  2669  		hrpc.NumberOfRows(1),
  2670  	)
  2671  	if err != nil {
  2672  		t.Fatalf("Failed to create scan request: %v", err)
  2673  	}
  2674  
  2675  	scanner := c.Scan(scan)
  2676  	defer scanner.Close()
  2677  	rsp, err := scanner.Next()
  2678  	if err != nil {
  2679  		t.Fatalf("Scanner.Next() returned error: %v", err)
  2680  	}
  2681  	if rsp == nil {
  2682  		t.Fatalf("Unexpected end of scanner")
  2683  	}
  2684  	expectedValue := []byte(strconv.Itoa(0))
  2685  	if !bytes.Equal(rsp.Cells[0].Value, expectedValue) {
  2686  		t.Errorf("Unexpected value. Got %v, want %v", rsp.Cells[0].Value, expectedValue)
  2687  	}
  2688  
  2689  	// force lease timeout
  2690  	time.Sleep(scannerLease * 2)
  2691  
  2692  	_, err = scanner.Next()
  2693  
  2694  	// lease timeout should return an UnknownScannerException
  2695  	if err != nil && strings.Contains(err.Error(),
  2696  		"org.apache.hadoop.hbase.UnknownScannerException") {
  2697  		t.Log("Error matches: UnknownScannerException")
  2698  	} else {
  2699  		t.Fatalf("Error does not match org.apache.hadoop.hbase.UnknownScannerException, "+
  2700  			"got: %v", err)
  2701  	}
  2702  }
  2703  
  2704  // TestScannerRenewal tests for the renewal process of scanners
  2705  // if the renew flag is enabled for a scan requset. If there is a long
  2706  // period of waiting between Next calls, the latter Next call should
  2707  // still succeed because we are renewing every lease timeout / 2 seconds
  2708  func TestScannerRenewal(t *testing.T) {
  2709  	c := gohbase.NewClient(*host)
  2710  	defer c.Close()
  2711  	// Insert test data
  2712  	keyPrefix := "scanner_renewal_test_"
  2713  	numRows := 2
  2714  	for i := 0; i < numRows; i++ {
  2715  		key := fmt.Sprintf("%s%d", keyPrefix, i)
  2716  		value := []byte(strconv.Itoa(i))
  2717  		err := insertKeyValue(c, key, "cf", value)
  2718  		if err != nil {
  2719  			t.Fatalf("Failed to insert test data: %v", err)
  2720  		}
  2721  	}
  2722  
  2723  	// Create a scan request
  2724  	// Turn on renewal
  2725  	// We set result size to 1 to force a lease timeout between Next calls
  2726  	scan, err := hrpc.NewScanStr(context.Background(), table,
  2727  		hrpc.Families(map[string][]string{"cf": nil}),
  2728  		hrpc.Filters(filter.NewPrefixFilter([]byte(keyPrefix))),
  2729  		hrpc.NumberOfRows(1),
  2730  		hrpc.RenewInterval(scannerLease/2),
  2731  	)
  2732  	if err != nil {
  2733  		t.Fatalf("Failed to create scan request: %v", err)
  2734  	}
  2735  
  2736  	scanner := c.Scan(scan)
  2737  	defer scanner.Close()
  2738  	for i := 0; i < numRows; i++ {
  2739  		rsp, err := scanner.Next()
  2740  		if err != nil {
  2741  			t.Fatalf("Scanner.Next() returned error: %v", err)
  2742  		}
  2743  		if rsp == nil {
  2744  			t.Fatalf("Unexpected end of scanner")
  2745  		}
  2746  		expectedValue := []byte(strconv.Itoa(i))
  2747  		if !bytes.Equal(rsp.Cells[0].Value, expectedValue) {
  2748  			t.Fatalf("Unexpected value. Got %v, want %v", rsp.Cells[0].Value, expectedValue)
  2749  		}
  2750  		// Sleep to trigger renewal
  2751  		time.Sleep(scannerLease * 2)
  2752  	}
  2753  	// Ensure scanner is exhausted
  2754  	rsp, err := scanner.Next()
  2755  	if err != io.EOF {
  2756  		t.Fatalf("Expected EOF error, got: %v", err)
  2757  	}
  2758  	if rsp != nil {
  2759  		t.Fatalf("Expected nil response at end of scan, got: %v", rsp)
  2760  	}
  2761  }
  2762  
  2763  func TestScannerRenewalCancellation(t *testing.T) {
  2764  	c := gohbase.NewClient(*host)
  2765  	defer c.Close()
  2766  
  2767  	// Insert test data
  2768  	keyPrefix := "scanner_renewal_cancel_test_"
  2769  	numRows := 2
  2770  	for i := 0; i < numRows; i++ {
  2771  		key := fmt.Sprintf("%s%d", keyPrefix, i)
  2772  		value := []byte(strconv.Itoa(i))
  2773  		err := insertKeyValue(c, key, "cf", value)
  2774  		if err != nil {
  2775  			t.Fatalf("Failed to insert test data: %v", err)
  2776  		}
  2777  	}
  2778  
  2779  	// Create a context with cancellation
  2780  	ctx, cancel := context.WithCancel(context.Background())
  2781  	defer cancel()
  2782  
  2783  	scan, err := hrpc.NewScanStr(ctx, table,
  2784  		hrpc.Families(map[string][]string{"cf": nil}),
  2785  		hrpc.Filters(filter.NewPrefixFilter([]byte(keyPrefix))),
  2786  		hrpc.NumberOfRows(1),
  2787  		hrpc.RenewInterval(scannerLease/2),
  2788  	)
  2789  	if err != nil {
  2790  		t.Fatalf("Failed to create scan request: %v", err)
  2791  	}
  2792  
  2793  	scanner := c.Scan(scan)
  2794  	defer scanner.Close()
  2795  
  2796  	rsp, err := scanner.Next()
  2797  	if err != nil {
  2798  		t.Fatalf("Scanner.Next() returned error: %v", err)
  2799  	}
  2800  	if rsp == nil {
  2801  		t.Fatalf("Unexpected end of scanner")
  2802  	}
  2803  	expectedValue := []byte(strconv.Itoa(0))
  2804  	if !bytes.Equal(rsp.Cells[0].Value, expectedValue) {
  2805  		t.Errorf("Unexpected value. Got %v, want %v", rsp.Cells[0].Value, expectedValue)
  2806  	}
  2807  
  2808  	// Cancel the context
  2809  	cancel()
  2810  
  2811  	// Next call should return an error
  2812  	_, err = scanner.Next()
  2813  
  2814  	if err == nil {
  2815  		t.Fatal("Expected error after context cancellation, got nil")
  2816  	}
  2817  	if err != context.Canceled {
  2818  		t.Fatalf("Expected context.Canceled error, got: %v", err)
  2819  	}
  2820  }
  2821  
  2822  func TestScanWithStatsHandler(t *testing.T) {
  2823  	ctx := context.Background()
  2824  	c := gohbase.NewClient(*host)
  2825  	defer c.Close()
  2826  
  2827  	// Prep data to scan over:
  2828  	q1, q2, q3 := "scanStats1", "scanStats2", "scanStats3"
  2829  	quals := []string{q1, q2, q3}
  2830  	fams := map[string][]string{"cf": quals}
  2831  	rows := 5
  2832  	for i := 0; i < rows; i++ {
  2833  		for _, q := range quals {
  2834  			err := insertKeyValueAtCol(c, strconv.Itoa(i), q, "cf", []byte(strconv.Itoa(i)),
  2835  				hrpc.Timestamp(time.UnixMilli(int64(i))))
  2836  			if err != nil {
  2837  				t.Fatalf("Failed to insert test data: %v", err)
  2838  			}
  2839  		}
  2840  	}
  2841  
  2842  	t.Run("Basic scan test that calls handler", func(t *testing.T) {
  2843  		ss := &hrpc.ScanStats{}
  2844  		tn := []byte("handler table update")
  2845  		h := func(stats *hrpc.ScanStats) {
  2846  			ss.Table = tn
  2847  			t.Logf("Handler called, ScanStats = %s", stats)
  2848  		}
  2849  
  2850  		scan, err := hrpc.NewScanRange(ctx, []byte(table), []byte(""), []byte("REVTEST-100"),
  2851  			hrpc.Families(fams),
  2852  			hrpc.WithScanStatsHandler(h))
  2853  		if err != nil {
  2854  			t.Fatal(err)
  2855  		}
  2856  
  2857  		_, err = consumeScanner(c, scan)
  2858  		if err != nil {
  2859  			t.Fatalf("Error consuming scanner: %v", err)
  2860  		}
  2861  
  2862  		if !bytes.Equal(ss.Table, tn) {
  2863  			t.Fatalf("expected table to be updated in ScanStats by handler to %v, got %v",
  2864  				tn, ss.Table)
  2865  		}
  2866  
  2867  		if ss.ScanMetrics != nil {
  2868  			t.Fatalf("expected ScanMetrics to be nil, got %v", ss.ScanMetrics)
  2869  		}
  2870  	})
  2871  
  2872  	t.Run("ScanStats with ScanMetrics", func(t *testing.T) {
  2873  		ss := &hrpc.ScanStats{}
  2874  		h := func(stats *hrpc.ScanStats) {
  2875  			ss = stats
  2876  			t.Logf("Handler called, ScanStats = %s", ss)
  2877  		}
  2878  
  2879  		scan, err := hrpc.NewScanRange(ctx, []byte(table),
  2880  			// The table has been pre-split into multiple regions.
  2881  			// For testability, the scan range is defined here to scan within one region, as an
  2882  			// open scan across multiple regions will result in multiple calls to SendRPC, calling
  2883  			// the handler each time and making it difficult to isolate
  2884  			[]byte(""), []byte("REVTEST-100"),
  2885  			hrpc.Families(fams),
  2886  			hrpc.TimeRange(time.UnixMilli(0), time.UnixMilli(5)),
  2887  			// Both WithStatsHandler and WithTrackScanMetrics need to be set to collect ScanMetrics
  2888  			// in ScanStats
  2889  			hrpc.WithScanStatsHandler(h),
  2890  			hrpc.TrackScanMetrics())
  2891  		if err != nil {
  2892  			t.Fatal(err)
  2893  		}
  2894  
  2895  		var rs []*hrpc.Result
  2896  		rs, err = consumeScanner(c, scan)
  2897  		if err != nil {
  2898  			t.Fatalf("Error consuming scanner: %v", err)
  2899  		}
  2900  
  2901  		if ss.ScanMetrics == nil {
  2902  			t.Fatal("expected ScanStats to have ScanMetrics set, but got nil")
  2903  		}
  2904  
  2905  		if ss.ScanMetrics["ROWS_SCANNED"]-ss.ScanMetrics["ROWS_FILTERED"] != int64(len(rs)) {
  2906  			t.Fatalf("Expected ScanMetrics to reflect %d rows returned to client.\n"+
  2907  				"ScanMetrics: %v,\nscan results: %v",
  2908  				len(rs), ss.ScanMetrics, rs)
  2909  		}
  2910  	})
  2911  
  2912  	t.Run("ScanStats with scan over multiple regions for ScanStatsID",
  2913  		func(t *testing.T) {
  2914  			statsRes := make([]*hrpc.ScanStats, 0)
  2915  			h := func(stats *hrpc.ScanStats) {
  2916  				t.Logf("Handler called, ScanStats = %s", stats)
  2917  				statsRes = append(statsRes, stats)
  2918  			}
  2919  
  2920  			// test table is pre-split to have multiple regions, so this scan will cover multiple
  2921  			// regions
  2922  			scan, err := hrpc.NewScan(ctx, []byte(table),
  2923  				hrpc.Families(fams),
  2924  				hrpc.WithScanStatsHandler(h))
  2925  			if err != nil {
  2926  				t.Fatal(err)
  2927  			}
  2928  
  2929  			_, err = consumeScanner(c, scan)
  2930  			if err != nil {
  2931  				t.Fatalf("Error consuming scanner: %v", err)
  2932  			}
  2933  
  2934  			// keySplits is used to split the table into pre-split regions. For this scan,
  2935  			// the number of ScanStats results (and hrpc.Results, although not tested here) should
  2936  			// equal the num regions in the table. The test is scanning a tiny amount of data
  2937  			// it has written to specified qualifiers, so wouldn't be hitting max result size, ex.
  2938  			if len(statsRes) != len(keySplits)+1 {
  2939  				t.Fatalf("Expected handler to be called %d times, got %d calls",
  2940  					len(keySplits)+1, len(statsRes))
  2941  			}
  2942  
  2943  			scanStatsID := statsRes[0].ScanStatsID
  2944  			for _, rs := range statsRes {
  2945  				if rs.ScanStatsID != scanStatsID {
  2946  					t.Fatalf("Expected ScanStatsID to be preserved in all sub-scans of"+
  2947  						"scanner, but was not - initial value %v, got %v",
  2948  						scanStatsID, rs.ScanStatsID)
  2949  				}
  2950  			}
  2951  		})
  2952  }
  2953  
  2954  // consumeScanner is a helper function to consume the scanner and return the result.
  2955  func consumeScanner(c gohbase.Client, scan *hrpc.Scan) ([]*hrpc.Result, error) {
  2956  	sc := c.Scan(scan)
  2957  	defer sc.Close()
  2958  
  2959  	rs := []*hrpc.Result{}
  2960  
  2961  	for {
  2962  		var r *hrpc.Result
  2963  		r, err := sc.Next()
  2964  		if err == io.EOF {
  2965  			break
  2966  		}
  2967  		if err != nil {
  2968  			return rs, err
  2969  		}
  2970  		rs = append(rs, r)
  2971  	}
  2972  
  2973  	return rs, nil
  2974  }