diff --git a/dev-support/git-jira-release-audit/.gitignore b/dev-support/git-jira-release-audit/.gitignore index ddab63162421..0771f36d02ee 100644 --- a/dev-support/git-jira-release-audit/.gitignore +++ b/dev-support/git-jira-release-audit/.gitignore @@ -2,3 +2,4 @@ *.log *.svg venv +new_for_*.csv diff --git a/dev-support/git-jira-release-audit/README.md b/dev-support/git-jira-release-audit/README.md index 896664e7b91d..396128ad55df 100644 --- a/dev-support/git-jira-release-audit/README.md +++ b/dev-support/git-jira-release-audit/README.md @@ -43,7 +43,7 @@ $ ./venv/bin/pip install -r ./requirements.txt Successfully installed... ``` -## Usage +## Basic Usage The tool provides basic help docs. @@ -54,6 +54,7 @@ usage: git_jira_release_audit.py [-h] [--populate-from-git POPULATE_FROM_GIT] [--db-path DB_PATH] [--initialize-db INITIALIZE_DB] [--report-new-for-release-line REPORT_NEW_FOR_RELEASE_LINE] + [--report-new-for-release-branch REPORT_NEW_FOR_RELEASE_BRANCH] [--git-repo-path GIT_REPO_PATH] [--remote-name REMOTE_NAME] [--development-branch DEVELOPMENT_BRANCH] @@ -61,7 +62,9 @@ usage: git_jira_release_audit.py [-h] [--populate-from-git POPULATE_FROM_GIT] [--release-line-regexp RELEASE_LINE_REGEXP] [--parse-release-tags PARSE_RELEASE_TAGS] [--fallback-actions-path FALLBACK_ACTIONS_PATH] - [--jira-url JIRA_URL] + [--jira-url JIRA_URL] --branch-1-fix-version + BRANCH_1_FIX_VERSION --branch-2-fix-version + BRANCH_2_FIX_VERSION optional arguments: -h, --help show this help message and exit @@ -69,90 +72,169 @@ optional arguments: Building the audit database: --populate-from-git POPULATE_FROM_GIT When true, populate the audit database from the Git - repository. + repository. (default: True) --populate-from-jira POPULATE_FROM_JIRA When true, populate the audit database from Jira. + (default: True) --db-path DB_PATH Path to the database file, or leave unspecified for a - transient db. + transient db. (default: audit.db) --initialize-db INITIALIZE_DB When true, initialize the database tables. This is destructive to the contents of an existing database. + (default: False) Generating reports: --report-new-for-release-line REPORT_NEW_FOR_RELEASE_LINE Builds a report of the Jira issues that are new on the target release line, not present on any of the associated release branches. (i.e., on branch-2 but - not branch-{2.0,2.1,...}) + not branch-{2.0,2.1,...}) (default: None) + --report-new-for-release-branch REPORT_NEW_FOR_RELEASE_BRANCH + Builds a report of the Jira issues that are new on the + target release branch, not present on any of the + previous release branches. (i.e., on branch-2.3 but + not branch-{2.0,2.1,...}) (default: None) Interactions with the Git repo: --git-repo-path GIT_REPO_PATH Path to the git repo, or leave unspecified to infer - from the current file's path. + from the current file's path. (default: + ./git_jira_release_audit.py) --remote-name REMOTE_NAME The name of the git remote to use when identifying - branches. Default: 'origin' + branches. Default: 'origin' (default: origin) --development-branch DEVELOPMENT_BRANCH The name of the branch from which all release lines - originate. Default: 'master' + originate. Default: 'master' (default: master) --development-branch-fix-version DEVELOPMENT_BRANCH_FIX_VERSION The Jira fixVersion used to indicate an issue is - committed to the development branch. Default: '3.0.0' + committed to the development branch. (default: 3.0.0) --release-line-regexp RELEASE_LINE_REGEXP - A regexp used to identify release lines. + A regexp used to identify release lines. (default: + branch-\d+$) --parse-release-tags PARSE_RELEASE_TAGS When true, look for release tags and annotate commits according to their release version. An Expensive - calculation, disabled by default. + calculation, disabled by default. (default: False) --fallback-actions-path FALLBACK_ACTIONS_PATH Path to a file containing _DB.Actions applicable to - specific git shas. + specific git shas. (default: fallback_actions.csv) --branch-1-fix-version BRANCH_1_FIX_VERSION The Jira fixVersion used to indicate an issue is committed to the specified release line branch + (default: None) --branch-2-fix-version BRANCH_2_FIX_VERSION The Jira fixVersion used to indicate an issue is committed to the specified release line branch + (default: None) Interactions with Jira: - --jira-url JIRA_URL A URL locating the target JIRA instance. + --jira-url JIRA_URL A URL locating the target JIRA instance. (default: + https://issues.apache.org/jira) ``` +### Build a Database + +This invocation will build a "simple" database, correlating commits to +branches. It omits gathering the detailed release tag data, so it runs pretty +quickly. + Example Run: ```shell script $ ./venv/bin/python3 ./git_jira_release_audit.py \ --db-path=audit.db \ - --remote-name=apache-rw \ --development-branch-fix-version=3.0.0 \ - --branch-1-fix-version=1.5.0 \ - --branch-2-fix-version=2.3.0 -INFO:root:apache-rw/branch-1 has 4046 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-1.0 has 1433 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-1.1 has 2111 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-1.2 has 2738 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-1.3 has 3287 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-1.4 has 3912 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. -INFO:root:apache-rw/branch-2 has 3080 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. -INFO:root:apache-rw/branch-2.0 has 2194 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. -INFO:root:apache-rw/branch-2.1 has 2705 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. -INFO:root:apache-rw/branch-2.2 has 2927 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. -INFO:root:retrieving 5653 jira_ids from the issue tracker - -apache-rw/branch-1 100%|██████████████████████████████████████████████████████| 4046/4046 [08:23<00:00, 8.04 commit/s] -apache-rw/branch-1.0 100%|████████████████████████████████████████████████████| 1433/1433 [03:49<00:00, 6.26 commit/s] -apache-rw/branch-1.1 100%|████████████████████████████████████████████████████| 2111/2111 [05:16<00:00, 6.68 commit/s] -apache-rw/branch-1.2 100%|████████████████████████████████████████████████████| 2738/2738 [06:26<00:00, 7.10 commit/s] -apache-rw/branch-1.3 100%|████████████████████████████████████████████████████| 3287/3287 [07:21<00:00, 7.46 commit/s] -apache-rw/branch-1.4 100%|████████████████████████████████████████████████████| 3912/3912 [08:08<00:00, 8.02 commit/s] -apache-rw/branch-2 100%|█████████████████████████████████████████████████████| 3080/3080 [03:29<00:00, 14.74 commit/s] -apache-rw/branch-2.0 100%|████████████████████████████████████████████████████| 2194/2194 [04:56<00:00, 7.42 commit/s] -apache-rw/branch-2.1 100%|███████████████████████████████████████████████████| 2705/2705 [03:17<00:00, 13.75 commit/s] -apache-rw/branch-2.2 100%|███████████████████████████████████████████████████| 2927/2927 [03:28<00:00, 14.09 commit/s] -fetch from Jira 100%|█████████████████████████████████████████████████████████| 5653/5653 [00:58<00:00, 98.29 issue/s] + --branch-1-fix-version=1.7.0 \ + --branch-2-fix-version=2.4.0 +INFO:git_jira_release_audit.py:origin/branch-1.0 has 1433 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. +INFO:git_jira_release_audit.py:origin/branch-1.1 has 2111 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. +INFO:git_jira_release_audit.py:origin/branch-1.2 has 2738 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. +INFO:git_jira_release_audit.py:origin/branch-1.3 has 3296 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. +INFO:git_jira_release_audit.py:origin/branch-1.4 has 3926 commits since its origin at 0167558eb31ff48308d592ef70b6d005ba6d21fb. +INFO:git_jira_release_audit.py:origin/branch-2 has 3325 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. +INFO:git_jira_release_audit.py:origin/branch-2.0 has 2198 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. +INFO:git_jira_release_audit.py:origin/branch-2.1 has 2749 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. +INFO:git_jira_release_audit.py:origin/branch-2.2 has 2991 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. +INFO:git_jira_release_audit.py:origin/branch-2.3 has 3312 commits since its origin at 0d0c330401ade938bf934aafd79ec23705edcc60. +INFO:git_jira_release_audit.py:retrieving 5850 jira_ids from the issue tracker + +origin/branch-1 100%|████████████████████████████████████| 4084/4084 [00:00<00:00, 9805.33 commit/s] +origin/branch-1.0 100%|█████████████████████████████████| 1433/1433 [00:00<00:00, 10479.89 commit/s] +origin/branch-1.1 100%|█████████████████████████████████| 2111/2111 [00:00<00:00, 10280.60 commit/s] +origin/branch-1.2 100%|██████████████████████████████████| 2738/2738 [00:00<00:00, 8833.51 commit/s] +origin/branch-1.3 100%|██████████████████████████████████| 3296/3296 [00:00<00:00, 9746.93 commit/s] +origin/branch-1.4 100%|██████████████████████████████████| 3926/3926 [00:00<00:00, 9750.96 commit/s] +origin/branch-2 100%|████████████████████████████████████| 3325/3325 [00:00<00:00, 9688.14 commit/s] +origin/branch-2.0 100%|██████████████████████████████████| 2198/2198 [00:00<00:00, 8804.18 commit/s] +origin/branch-2.1 100%|██████████████████████████████████| 2749/2749 [00:00<00:00, 9328.67 commit/s] +origin/branch-2.2 100%|██████████████████████████████████| 2991/2991 [00:00<00:00, 9215.56 commit/s] +origin/branch-2.3 100%|██████████████████████████████████| 3312/3312 [00:00<00:00, 9063.19 commit/s] +fetch from Jira 100%|████████████████████████████████████████| 5850/5850 [10:40<00:00, 9.14 issue/s] +``` + +Optionally, the database can be build to include release tags, by specifying +`--parse-release-tags=true`. This is more time-consuming, but is necessary for +auditing discrepancies between git and Jira. Running the same command but +including this flag looks like this: + +```shell script +origin/branch-1 100%|███████████████████████████████████████| 4084/4084 [08:58<00:00, 7.59 commit/s] +origin/branch-1.0 100%|█████████████████████████████████████| 1433/1433 [03:54<00:00, 6.13 commit/s] +origin/branch-1.1 100%|█████████████████████████████████████| 2111/2111 [41:26<00:00, 0.85 commit/s] +origin/branch-1.2 100%|█████████████████████████████████████| 2738/2738 [07:10<00:00, 6.37 commit/s] +origin/branch-1.3 100%|██████████████████████████████████| 3296/3296 [2h 33:13<00:00, 0.36 commit/s] +origin/branch-1.4 100%|██████████████████████████████████| 3926/3926 [7h 22:41<00:00, 0.15 commit/s] +origin/branch-2 100%|████████████████████████████████████| 3325/3325 [2h 05:43<00:00, 0.44 commit/s] +origin/branch-2.0 100%|█████████████████████████████████████| 2198/2198 [52:18<00:00, 0.70 commit/s] +origin/branch-2.1 100%|█████████████████████████████████████| 2749/2749 [17:09<00:00, 2.67 commit/s] +origin/branch-2.2 100%|█████████████████████████████████████| 2991/2991 [52:15<00:00, 0.95 commit/s] +origin/branch-2.3 100%|████████████████████████████████████| 3312/3312 [05:08<00:00, 10.74 commit/s] +fetch from Jira 100%|████████████████████████████████████████| 5850/5850 [10:46<00:00, 9.06 issue/s] +``` + +### Run a Report + +With a database populated with branch information, the build-in reports can be +run. + +`--report-new-for-release-line` +> Builds a report of the Jira issues that are new on the target release line, +not present on any of the associated release branches. (i.e., on branch-2 but +not branch-{2.0,2.1,...}) + +`--report-new-for-release-branch` +> Builds a report of the Jira issues that are new on the target release branch, +not present on any of the previous release branches. (i.e., on branch-2.3 but +not branch-{2.0,2.1,...}) + +Either way, the output is a csv file containing a summary of each JIRA id found +matching the report criteria. + +Example Run: + +```shell script +$ ./venv/bin/python3.7 ./git_jira_release_audit.py \ + --populate-from-git=false \ + --populate-from-jira=false \ + --branch-1-fix-version=1.7.0 \ + --branch-2-fix-version=2.4.0 \ + --report-new-for-release-branch=origin/branch-2.3 +INFO:git_jira_release_audit.py:retrieving 292 jira_ids from the issue tracker +INFO:git_jira_release_audit.py:generated report at new_for_origin-branch-2.3.csv + +fetch from Jira 100%|████████████████████████████████████████| 292/292 [00:03<00:00, 114.01 issue/s] +$ head -n5 new_for_origin-branch-2.3.csv +key,issue_type,priority,summary,resolution,components +HBASE-21070,Bug,Critical,SnapshotFileCache won't update for snapshots stored in S3,Fixed,['snapshots'] +HBASE-21773,Bug,Critical,rowcounter utility should respond to pleas for help,Fixed,['tooling'] +HBASE-21505,Bug,Major,Several inconsistencies on information reported for Replication Sources by hbase shell status 'replication' command.,Fixed,['Replication'] +HBASE-22057,Bug,Major,Impose upper-bound on size of ZK ops sent in a single multi(),Fixed,[] ``` -With a populated database, query with sqlite: +### Explore the Database + +With a populated database, query it with sqlite: ```shell script $ sqlite3 audit.db @@ -160,56 +242,71 @@ SQLite version 3.24.0 2018-06-04 14:10:15 Enter ".help" for usage hints. sqlite> -- count the number of distinct commits on a release branch sqlite> select count(distinct jira_id), branch from git_commits group by branch; -3406|apache-rw/branch-1 -1189|apache-rw/branch-1.0 -1728|apache-rw/branch-1.1 -2289|apache-rw/branch-1.2 -2779|apache-rw/branch-1.3 -3277|apache-rw/branch-1.4 -2666|apache-rw/branch-2 -1809|apache-rw/branch-2.0 -2289|apache-rw/branch-2.1 -2511|apache-rw/branch-2.2 - -sqlite> -- count the number of issues that will be in 2.3.0 that have not been released on any earlier -sqlite> -- version. -sqlite> select count(1) from ( - select distinct jira_id from git_commits where branch = 'apache-rw/branch-2' except - select distinct jira_id from git_commits where branch in - ('apache-rw/branch-2.0', 'apache-rw/branch-2.1', 'apache-rw/branch-2.2')); -169 +3437|origin/branch-1 +1189|origin/branch-1.0 +1728|origin/branch-1.1 +2289|origin/branch-1.2 +2788|origin/branch-1.3 +3289|origin/branch-1.4 +2846|origin/branch-2 +1813|origin/branch-2.0 +2327|origin/branch-2.1 +2566|origin/branch-2.2 +2839|origin/branch-2.3 sqlite> -- find the issues for which the git commit record and JIRA fixVersion disagree +sqlite> -- this query requires the database be built with --parse-release-tags sqlite> select g.jira_id, g.git_tag, j.fix_version from git_commits g inner join jira_versions j on g.jira_id = j.jira_id - and g.branch = 'apache-rw/branch-2.2' + and g.branch = 'origin/branch-2.2' and g.git_tag is not null and j.fix_version like '2.2.%' and g.git_tag != j.fix_version; HBASE-22941|2.2.2|2.2.1 -sqlite> -- show jira non-1.x fixVersions for all issues on branch-2 but not on any -sqlite> -- branch-2.x release branch; i.e., issues that are missing a fixVersion or -sqlite> -- are marked for a release other than (3.0.0, 2.3.0) -sqlite> select g.jira_id, j.fix_version -from ( - select distinct jira_id from git_commits where branch = 'apache-rw/branch-2' except - select distinct jira_id from git_commits where branch in - (select distinct branch from git_commits where branch like 'apache-rw/branch-2.%')) g -left join jira_versions j - on g.jira_id = j.jira_id - and j.fix_version not like '1.%' -where ( - j.fix_version is null - OR j.fix_version not in ('3.0.0', '2.3.0')) -order by g.jira_id desc; -HBASE-23683|2.2.4 +sqlite> -- show jira fixVersions for all issues on branch-2.3 but not on any earlier +sqlite> -- branch; i.e., issues that are missing a fixVersion or are marked for +sqlite> -- a release other than the expected (3.0.0, 2.3.0). +sqlite> -- this query requires the database be built with --parse-release-tags +sqlite> select jira_id, fix_version + FROM jira_versions + WHERE jira_id in ( + SELECT distinct jira_id + FROM git_commits + WHERE branch = 'origin/branch-2.3' + EXCEPT SELECT distinct jira_id + FROM git_commits + WHERE branch IN ( + SELECT distinct branch + FROM git_commits + WHERE branch != 'origin/branch-2.3')) + AND fix_version NOT IN ('3.0.0', '2.3.0') + ORDER BY jira_id; +HBASE-22321|1.5.0 +HBASE-22360|2.2.0 +HBASE-22405|2.2.0 +HBASE-22555|2.4.0 HBASE-23032|connector-1.0.1 HBASE-23032|hbase-filesystem-1.0.0-alpha2 -HBASE-22405|2.2.0 -HBASE-22360|2.2.0 -HBASE-22321| -HBASE-22283|2.2.0 +HBASE-23604|HBASE-18095 +HBASE-23633|2.4.0 +HBASE-23647|HBASE-18095 +HBASE-23648|HBASE-18095 +HBASE-23731|HBASE-18095 +HBASE-23741|2.4.0 +HBASE-23752|HBASE-18095 +HBASE-23804|HBASE-18095 +HBASE-23851|master +HBASE-23936|2.4.0 +HBASE-23937|2.4.0 +HBASE-23977|2.4.0 +HBASE-24002|2.4.0 +HBASE-24033|2.4.0 +HBASE-24037|2.4.0 +HBASE-24073|master +HBASE-24075|2.4.0 +HBASE-24080|2.4.0 +HBASE-24080|master ``` diff --git a/dev-support/git-jira-release-audit/fallback_actions.csv b/dev-support/git-jira-release-audit/fallback_actions.csv index 0604eeb21e4e..eb6c97c1c567 100644 --- a/dev-support/git-jira-release-audit/fallback_actions.csv +++ b/dev-support/git-jira-release-audit/fallback_actions.csv @@ -27,8 +27,10 @@ hexsha,action,jira_id 05f8e94191ef6a63baadf56d6114d7d0317796f2,SKIP, 0791b878422eadf00b55076338f09bf059f39f0c,SKIP, 07f9f3d38cf4d0d01044ab28d90a50a1a009f6b8,SKIP, +0bff1305134b9c3a0bcad21900f5af68a8aedb4a,SKIP, 10f00547627076d79d77cf58dd2deaece2287084,ADD,HBASE-22330 10f3b77748a02a2c11635c33964929c0474e890d,SKIP, +1196e42362312080d3c523c107b5e8fefef9e57e,SKIP, 1404d5a97331ecc63db53971f5cb7329cb40ce67,ADD,HBASE-15203 14a869828fe481697d29b2d6e4135e8026039a38,SKIP, 1546613e76b1013a08ebc179c2c22bfeb44f3a4a,SKIP, @@ -42,6 +44,7 @@ hexsha,action,jira_id 1b3557649c9ee682c7f135ca52a0e3cd10cb9219,SKIP, 1c46250bef9ef9be9c255d61bda69ff7792ed551,SKIP, 1cb7d0e82ad64f37fbd6de950b74081b0d5eddf3,SKIP, +1d988afc9d2065a51fe74d0553f0943ef540dfaa,SKIP, 1eaef185327171b3dd3edb303e08cfe85186e745,SKIP, 1eb8ac6fe9dd0c15cdb52f66ced4136316c06465,SKIP, 2068804d7510e8c1f822b5db3cd4585455f6e7e7,SKIP, @@ -52,6 +55,7 @@ hexsha,action,jira_id 259d12f7397679c6b0d0a4788e5a37f65fd49f20,SKIP, 267bce0590c39570ddb935921e34bda35e3aa44c,SKIP, 278828333c44493ccbaa7db26a788b2756632034,SKIP, +27c1f2f978142b7bb4135e53c3ae067c5608c9fb,SKIP, 288794d68ba5bd4d1fd8d5c315cee972019dcb3d,ADD,HBASE-22330 28f07451a5dddf0ab3988b32b8672654fdbc5b58,SKIP, 2ba542d74c2d9e78332c8c94289d1295752d8072,SKIP, @@ -97,7 +101,9 @@ hexsha,action,jira_id 4eb84651a2b6d02d2074143308cef5d0f4b856a3,SKIP, 4f5b22bc19cb8d24ced5d42ebd9794cfd83bae85,SKIP, 54337870eda5649ab7bb81ed01c9dd25d59204f2,SKIP, +558ee079fd04dfab8e61eca10ee98ab5bac89dfa,SKIP, 58ab201be341f02829286f036a7401d0806eb999,SKIP, +58b63e04c4af99a5730efb0a7e553be4d950e6a5,SKIP, 5a16c15d7f51087a50511a2e0730f547c97a033f,SKIP, 5b5ff1d8b2cc43f78acaf9bc960be382dc6c34f7,SKIP, 5fa15dd7488433ea610ff5e92161409d20565690,SKIP, @@ -170,6 +176,7 @@ b3d55441b8174c704ada4585603f6bcfca298843,SKIP, b65231d04dbc565a578ce928e809aa51f5439857,SKIP, b6549007b313e8f3aa993d5c1ebd29c84ccb7b7b,SKIP, b6d4fc955fe0fc41f5225f1cc2e3e4b92029251c,SKIP, +b9c676cdc048c52f927cfa906fd18ff412e4ca20,SKIP, b9f5c6b065ebd572193c1fdc9d38557320b42fe6,SKIP, bcadcef21048e4764f7ae8dec3ce52884f20c02c,SKIP, bcdc56ac76e4a26e53faa8301a441e94ee8614d7,SKIP, @@ -178,9 +185,11 @@ bd4e14db07ea32a45c3ef734e06d195a405da67c,SKIP, bd4eba2b53b7af738fd9584511d737c4393d0855,SKIP, bef0616ef33306afca3060b96c2cba5f9762035d,SKIP, c100fb835a54be6002fe9704349e726f27b15b7a,SKIP, +c5e0a1397b3c6a14612e4c5b66f995c02de4310b,SKIP, c71da858ada94e1b93065f0b7caf3558942bc4da,SKIP, c89cfd3406823cf05fa83464c5ddee16bf0d473f,ADD,HBASE-17248 c89cfd3406823cf05fa83464c5ddee16bf0d473f,ADD,HBASE-17248 +c8c2a875056f27c9af81293a504d75634cbc1fa5,SKIP, c97905a962b88a0c68ca8a51c2e507daec81ca6d,SKIP, c9f506a2973e0acbd0d2df7b9353c9291f6c94a8,SKIP, cbb2c7e00d0c0b3f641250d981b9c87286d31058,ADD,HBASE-23069 @@ -209,6 +218,7 @@ e40fcee6b54712b76d702af6937c3320c60df2b9,SKIP, e501fe1a296be8fec0890e7e15414683aa3d933b,SKIP, e5349d589c000e395e12340e003aa9e2153afea6,SKIP, e5fb8214b2bfd6396539a4e8b6cf5f3cc5e9c06f,REVERT,HBASE-21874 +e869a20123afe326e198d35d110f5c0360ea244f,SKIP, e8e45ef8f2fb91a870399636b492d5cee58a4c39,SKIP, e92a147e1961366e36a39577816994566e1e21c5,SKIP, eacf3cb29641af1a68978d9bd7654f643a3aa3a1,SKIP, diff --git a/dev-support/git-jira-release-audit/git_jira_release_audit.py b/dev-support/git-jira-release-audit/git_jira_release_audit.py index 7eac6d33d474..db2788d081d0 100644 --- a/dev-support/git-jira-release-audit/git_jira_release_audit.py +++ b/dev-support/git-jira-release-audit/git_jira_release_audit.py @@ -30,12 +30,16 @@ import re import sqlite3 import time +import os import enlighten import git import jira +LOG = logging.getLogger(os.path.basename(__file__)) + + class _DB: """Manages an instance of Sqlite on behalf of the application. @@ -46,6 +50,9 @@ class _DB: Attributes: conn (:obj:`sqlite3.db2api.Connection`): The underlying connection object. """ + + SQL_LOG = LOG.getChild("sql") + class Action(enum.Enum): """Describes an action to be taken against the database.""" ADD = 'ADD' @@ -54,6 +61,7 @@ class Action(enum.Enum): def __init__(self, db_path, initialize_db, **_kwargs): self._conn = sqlite3.connect(db_path) + self._conn.set_trace_callback(_DB.log_query) if initialize_db: for table in 'git_commits', 'jira_versions': @@ -81,6 +89,10 @@ def __enter__(self): def __exit__(self, exc_type, exc_val, exc_tb): self._conn.close() + @staticmethod + def log_query(query): + _DB.SQL_LOG.debug(re.sub(r'\s+', ' ', query).strip()) + @property def conn(self): """:obj:`sqlite3.db2api.Connection`: Underlying database handle.""" @@ -324,7 +336,7 @@ def _set_release_tag(self, branch, tag, shas): def _resolve_ambiguity(self, commit): if commit.hexsha not in self._fallback_actions: - logging.warning('Unable to resolve action for %s: %s', commit.hexsha, commit.summary) + LOG.warning('Unable to resolve action for %s: %s', commit.hexsha, commit.summary) return _DB.Action.SKIP, None action, jira_id = self._fallback_actions[commit.hexsha] if not jira_id: @@ -354,7 +366,7 @@ def populate_db_release_branch(self, origin_commit, release_branch): global MANAGER commits = list(self._repo.iter_commits( "%s...%s" % (origin_commit.hexsha, release_branch), reverse=True)) - logging.info("%s has %d commits since its origin at %s.", release_branch, len(commits), + LOG.info("%s has %d commits since its origin at %s.", release_branch, len(commits), origin_commit) counter = MANAGER.counter(total=len(commits), desc=release_branch, unit='commit') commits_since_release = list() @@ -404,7 +416,7 @@ def populate_db(self): database.""" global MANAGER jira_ids = self._db.unique_jira_ids_from_git() - logging.info("retrieving %s jira_ids from the issue tracker", len(jira_ids)) + LOG.info("retrieving %s jira_ids from the issue tracker", len(jira_ids)) counter = MANAGER.counter(total=len(jira_ids), desc='fetch from Jira', unit='issue') chunk_size = 50 chunks = [jira_ids[i:i + chunk_size] for i in range(0, len(jira_ids), chunk_size)] @@ -429,7 +441,7 @@ def populate_db(self): def fetch_issues(self, jira_ids): """Retrieve the specified jira Ids.""" global MANAGER - logging.info("retrieving %s jira_ids from the issue tracker", len(jira_ids)) + LOG.info("retrieving %s jira_ids from the issue tracker", len(jira_ids)) counter = MANAGER.counter(total=len(jira_ids), desc='fetch from Jira', unit='issue') chunk_size = 50 chunks = [jira_ids[i:i + chunk_size] for i in range(0, len(jira_ids), chunk_size)] @@ -501,17 +513,17 @@ def _write_report(filename, issues): writer.writeheader() for issue in issues: writer.writerow(issue) - logging.info('generated report at %s', filename) + LOG.info('generated report at %s', filename) def report_new_for_release_line(self, release_line): """Builds a report of the Jira issues that are new on the target release line, not present on any of the associated release branches. (i.e., on branch-2 but not branch-{2.0,2.1,...})""" matches = [x for x in self._repo_reader.release_line_refs - if x.name == release_line or x.name.endswith('/%s' % release_line)] + if x.name == release_line or x.remote_head == release_line] release_line_ref = next(iter(matches), None) if not release_line_ref: - logging.error('release line %s not found. available options are %s.', + LOG.error('release line %s not found. available options are %s.', release_line, [x.name for x in self._repo_reader.release_line_refs]) return cursor = self._db.conn.execute(""" @@ -525,6 +537,31 @@ def report_new_for_release_line(self, release_line): filename = 'new_for_%s.csv' % release_line.replace('/', '-') Auditor._write_report(filename, issues) + def report_new_for_release_branch(self, release_branch): + """Builds a report of the Jira issues that are new on the target release branch, not present + on any of the previous release branches. (i.e., on branch-2.3 but not + branch-{2.0,2.1,...})""" + matches = [x for x in self._repo_reader.release_branch_refs + if x.name == release_branch or x.remote_head == release_branch] + release_branch_ref = next(iter(matches), None) + if not release_branch_ref: + LOG.error('release branch %s not found. available options are %s.', + release_branch, [x.name for x in self._repo_reader.release_branch_refs]) + return + previous_branches = [x.name for x in self._repo_reader.release_branch_refs + if x.remote_head != release_branch_ref.remote_head] + query = ( + "SELECT distinct jira_id FROM git_commits" + " WHERE branch = ?" + " EXCEPT SELECT distinct jira_id FROM git_commits" + f" WHERE branch IN ({','.join('?' for _ in previous_branches)})" + ) + cursor = self._db.conn.execute(query, tuple([release_branch_ref.name] + previous_branches)) + jira_ids = [x[0] for x in cursor.fetchall()] + issues = self._jira_reader.fetch_issues(jira_ids) + filename = 'new_for_%s.csv' % release_branch.replace('/', '-') + Auditor._write_report(filename, issues) + @staticmethod def _str_to_bool(val): if not val: @@ -548,7 +585,7 @@ def _build_first_pass_parser(): building_group.add_argument( '--db-path', help='Path to the database file, or leave unspecified for a transient db.', - default=':memory:') + default='audit.db') building_group.add_argument( '--initialize-db', help='When true, initialize the database tables. This is destructive to the contents' @@ -561,6 +598,11 @@ def _build_first_pass_parser(): help=Auditor.report_new_for_release_line.__doc__, type=str, default=None) + report_group.add_argument( + '--report-new-for-release-branch', + help=Auditor.report_new_for_release_branch.__doc__, + type=str, + default=None) git_repo_group = parser.add_argument_group('Interactions with the Git repo') git_repo_group.add_argument( '--git-repo-path', @@ -580,7 +622,7 @@ def _build_first_pass_parser(): git_repo_group.add_argument( '--development-branch-fix-version', help='The Jira fixVersion used to indicate an issue is committed to the development' - + ' branch. Default: \'3.0.0\'', + + ' branch.', default='3.0.0') git_repo_group.add_argument( '--release-line-regexp', @@ -612,7 +654,10 @@ def _build_second_pass_parser(repo_reader, parent_parser, git_repo_group): help='The Jira fixVersion used to indicate an issue is committed to the specified ' + 'release line branch', required=True) - return argparse.ArgumentParser(parents=[parent_parser]) + return argparse.ArgumentParser( + parents=[parent_parser], + formatter_class=argparse.ArgumentDefaultsHelpFormatter + ) MANAGER = None @@ -621,11 +666,11 @@ def _build_second_pass_parser(repo_reader, parent_parser, git_repo_group): def main(): global MANAGER + logging.basicConfig(level=logging.INFO) first_pass_parser, git_repo_group = Auditor._build_first_pass_parser() first_pass_args, extras = first_pass_parser.parse_known_args() first_pass_args_dict = vars(first_pass_args) with _DB(**first_pass_args_dict) as db: - logging.basicConfig(level=logging.INFO) repo_reader = _RepoReader(db, **first_pass_args_dict) jira_reader = _JiraReader(db, **first_pass_args_dict) second_pass_parser = Auditor._build_second_pass_parser( @@ -641,6 +686,9 @@ def main(): if second_pass_args.report_new_for_release_line: release_line = second_pass_args.report_new_for_release_line auditor.report_new_for_release_line(release_line) + if second_pass_args.report_new_for_release_branch: + release_branch = second_pass_args.report_new_for_release_branch + auditor.report_new_for_release_branch(release_branch) if __name__ == '__main__': diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java index f85fbefedb87..2aadc4257c7c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java @@ -915,7 +915,7 @@ public static List getTags(Cell cell) { * Retrieve Cell's first tag, matching the passed in type * @param cell The Cell * @param type Type of the Tag to retrieve - * @return null if there is no tag of the passed in tag type + * @return Optional, empty if there is no tag of the passed in tag type */ public static Optional getTag(Cell cell, byte type) { boolean bufferBacked = cell instanceof ByteBufferExtendedCell; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index f096ef9a7ab6..b659d141f931 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -24,9 +24,12 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; +import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + /** * Immutable POJO class for representing a table name. * Which is of the form: @@ -146,9 +149,7 @@ public static boolean isMetaTableName(final TableName tn) { throw new IllegalArgumentException("Name is null or empty"); } - int namespaceDelimIndex = - org.apache.hbase.thirdparty.com.google.common.primitives.Bytes.lastIndexOf(tableName, - (byte) NAMESPACE_DELIM); + int namespaceDelimIndex = ArrayUtils.lastIndexOf(tableName, (byte) NAMESPACE_DELIM); if (namespaceDelimIndex < 0){ isLegalTableQualifierName(tableName); } else { @@ -433,33 +434,73 @@ public static TableName valueOf(String namespaceAsString, String qualifierAsStri /** + * @param fullName will use the entire byte array * @throws IllegalArgumentException if fullName equals old root or old meta. Some code * depends on this. The test is buried in the table creation to save on array comparison * when we're creating a standard table object that will be in the cache. */ public static TableName valueOf(byte[] fullName) throws IllegalArgumentException{ + return valueOf(fullName, 0, fullName.length); + } + + /** + * @param fullName byte array to look into + * @param offset within said array + * @param length within said array + * @throws IllegalArgumentException if fullName equals old root or old meta. + */ + public static TableName valueOf(byte[] fullName, int offset, int length) + throws IllegalArgumentException { + Preconditions.checkArgument(offset >= 0, "offset must be non-negative but was %s", offset); + Preconditions.checkArgument(offset < fullName.length, "offset (%s) must be < array length (%s)", + offset, fullName.length); + Preconditions.checkArgument(length <= fullName.length, + "length (%s) must be <= array length (%s)", length, fullName.length); for (TableName tn : tableCache) { - if (Arrays.equals(tn.getName(), fullName)) { + final byte[] tnName = tn.getName(); + if (Bytes.equals(tnName, 0, tnName.length, fullName, offset, length)) { return tn; } } - int namespaceDelimIndex = - org.apache.hbase.thirdparty.com.google.common.primitives.Bytes.lastIndexOf(fullName, - (byte) NAMESPACE_DELIM); + int namespaceDelimIndex = ArrayUtils.lastIndexOf(fullName, (byte) NAMESPACE_DELIM, + offset + length - 1); - if (namespaceDelimIndex < 0) { + if (namespaceDelimIndex < offset) { return createTableNameIfNecessary( ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME), - ByteBuffer.wrap(fullName)); + ByteBuffer.wrap(fullName, offset, length)); } else { return createTableNameIfNecessary( - ByteBuffer.wrap(fullName, 0, namespaceDelimIndex), - ByteBuffer.wrap(fullName, namespaceDelimIndex + 1, - fullName.length - (namespaceDelimIndex + 1))); + ByteBuffer.wrap(fullName, offset, namespaceDelimIndex), + ByteBuffer.wrap(fullName, namespaceDelimIndex + 1, length - (namespaceDelimIndex + 1))); } } + /** + * @param fullname of a table, possibly with a leading namespace and ':' as delimiter. + * @throws IllegalArgumentException if fullName equals old root or old meta. + */ + public static TableName valueOf(ByteBuffer fullname) { + fullname = fullname.duplicate(); + fullname.mark(); + boolean miss = true; + while (fullname.hasRemaining() && miss) { + miss = ((byte) NAMESPACE_DELIM) != fullname.get(); + } + if (miss) { + fullname.reset(); + return valueOf(null, fullname); + } else { + ByteBuffer qualifier = fullname.slice(); + int delimiterIndex = fullname.position() - 1; + fullname.reset(); + // changing variable name for clarity + ByteBuffer namespace = fullname.duplicate(); + namespace.limit(delimiterIndex); + return valueOf(namespace, qualifier); + } + } /** * @throws IllegalArgumentException if fullName equals old root or old meta. Some code diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java index 7771c0a42993..50d8dffaac47 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -19,7 +19,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -69,8 +68,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - @Category({ ReplicationTests.class, LargeTests.class }) public class TestVerifyReplication extends TestReplicationBase { @@ -108,7 +105,7 @@ public static void setUpBeforeClass() throws Exception { htable3 = connection2.getTable(peerTableName); } - private void runVerifyReplication(String[] args, int expectedGoodRows, int expectedBadRows) + static void runVerifyReplication(String[] args, int expectedGoodRows, int expectedBadRows) throws IOException, InterruptedException, ClassNotFoundException { Job job = new VerifyReplication().createSubmittableJob(new Configuration(CONF1), args); if (job == null) { @@ -240,185 +237,7 @@ public void testVerifyRepJobWithRawOptions() throws Exception { } } - // VerifyReplication should honor versions option - @Test - public void testHBase14905() throws Exception { - // normal Batch tests - byte[] qualifierName = Bytes.toBytes("f1"); - Put put = new Put(Bytes.toBytes("r1")); - long ts = System.currentTimeMillis(); - put.addColumn(famName, qualifierName, ts + 1, Bytes.toBytes("v1002")); - htable1.put(put); - put.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v1001")); - htable1.put(put); - put.addColumn(famName, qualifierName, ts + 3, Bytes.toBytes("v1112")); - htable1.put(put); - - Scan scan = new Scan(); - scan.readVersions(100); - ResultScanner scanner1 = htable1.getScanner(scan); - Result[] res1 = scanner1.next(1); - scanner1.close(); - - assertEquals(1, res1.length); - assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); - - for (int i = 0; i < NB_RETRIES; i++) { - scan = new Scan(); - scan.readVersions(100); - scanner1 = htable2.getScanner(scan); - res1 = scanner1.next(1); - scanner1.close(); - if (res1.length != 1) { - LOG.info("Only got " + res1.length + " rows"); - Thread.sleep(SLEEP_TIME); - } else { - int cellNumber = res1[0].getColumnCells(famName, Bytes.toBytes("f1")).size(); - if (cellNumber != 3) { - LOG.info("Only got " + cellNumber + " cells"); - Thread.sleep(SLEEP_TIME); - } else { - break; - } - } - if (i == NB_RETRIES - 1) { - fail("Waited too much time for normal batch replication"); - } - } - - put.addColumn(famName, qualifierName, ts + 4, Bytes.toBytes("v1111")); - htable2.put(put); - put.addColumn(famName, qualifierName, ts + 5, Bytes.toBytes("v1112")); - htable2.put(put); - - scan = new Scan(); - scan.readVersions(100); - scanner1 = htable2.getScanner(scan); - res1 = scanner1.next(NB_ROWS_IN_BATCH); - scanner1.close(); - - assertEquals(1, res1.length); - assertEquals(5, res1[0].getColumnCells(famName, qualifierName).size()); - - String[] args = new String[] { "--versions=100", PEER_ID, tableName.getNameAsString() }; - runVerifyReplication(args, 0, 1); - } - - // VerifyReplication should honor versions option - @Test - public void testVersionMismatchHBase14905() throws Exception { - // normal Batch tests - byte[] qualifierName = Bytes.toBytes("f1"); - Put put = new Put(Bytes.toBytes("r1")); - long ts = System.currentTimeMillis(); - put.addColumn(famName, qualifierName, ts + 1, Bytes.toBytes("v1")); - htable1.put(put); - put.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v2")); - htable1.put(put); - put.addColumn(famName, qualifierName, ts + 3, Bytes.toBytes("v3")); - htable1.put(put); - - Scan scan = new Scan(); - scan.readVersions(100); - ResultScanner scanner1 = htable1.getScanner(scan); - Result[] res1 = scanner1.next(1); - scanner1.close(); - - assertEquals(1, res1.length); - assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); - - for (int i = 0; i < NB_RETRIES; i++) { - scan = new Scan(); - scan.readVersions(100); - scanner1 = htable2.getScanner(scan); - res1 = scanner1.next(1); - scanner1.close(); - if (res1.length != 1) { - LOG.info("Only got " + res1.length + " rows"); - Thread.sleep(SLEEP_TIME); - } else { - int cellNumber = res1[0].getColumnCells(famName, Bytes.toBytes("f1")).size(); - if (cellNumber != 3) { - LOG.info("Only got " + cellNumber + " cells"); - Thread.sleep(SLEEP_TIME); - } else { - break; - } - } - if (i == NB_RETRIES - 1) { - fail("Waited too much time for normal batch replication"); - } - } - - try { - // Disabling replication and modifying the particular version of the cell to validate the - // feature. - hbaseAdmin.disableReplicationPeer(PEER_ID); - Put put2 = new Put(Bytes.toBytes("r1")); - put2.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v99")); - htable2.put(put2); - - scan = new Scan(); - scan.readVersions(100); - scanner1 = htable2.getScanner(scan); - res1 = scanner1.next(NB_ROWS_IN_BATCH); - scanner1.close(); - assertEquals(1, res1.length); - assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); - - String[] args = new String[] { "--versions=100", PEER_ID, tableName.getNameAsString() }; - runVerifyReplication(args, 0, 1); - } finally { - hbaseAdmin.enableReplicationPeer(PEER_ID); - } - } - - @Test - public void testVerifyReplicationPrefixFiltering() throws Exception { - final byte[] prefixRow = Bytes.toBytes("prefixrow"); - final byte[] prefixRow2 = Bytes.toBytes("secondrow"); - loadData("prefixrow", prefixRow); - loadData("secondrow", prefixRow2); - loadData("aaa", row); - loadData("zzz", row); - waitForReplication(NB_ROWS_IN_BATCH * 4, NB_RETRIES * 4); - String[] args = - new String[] { "--row-prefixes=prefixrow,secondrow", PEER_ID, tableName.getNameAsString() }; - runVerifyReplication(args, NB_ROWS_IN_BATCH * 2, 0); - } - - @Test - public void testVerifyReplicationSnapshotArguments() { - String[] args = - new String[] { "--sourceSnapshotName=snapshot1", "2", tableName.getNameAsString() }; - assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--sourceSnapshotTmpDir=tmp", "2", tableName.getNameAsString() }; - assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--sourceSnapshotName=snapshot1", "--sourceSnapshotTmpDir=tmp", "2", - tableName.getNameAsString() }; - assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--peerSnapshotName=snapshot1", "2", tableName.getNameAsString() }; - assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--peerSnapshotTmpDir=/tmp/", "2", tableName.getNameAsString() }; - assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--peerSnapshotName=snapshot1", "--peerSnapshotTmpDir=/tmp/", - "--peerFSAddress=tempfs", "--peerHBaseRootAddress=hdfs://tempfs:50070/hbase/", "2", - tableName.getNameAsString() }; - assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - - args = new String[] { "--sourceSnapshotName=snapshot1", "--sourceSnapshotTmpDir=/tmp/", - "--peerSnapshotName=snapshot2", "--peerSnapshotTmpDir=/tmp/", "--peerFSAddress=tempfs", - "--peerHBaseRootAddress=hdfs://tempfs:50070/hbase/", "2", tableName.getNameAsString() }; - - assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); - } - - private void checkRestoreTmpDir(Configuration conf, String restoreTmpDir, int expectedCount) + static void checkRestoreTmpDir(Configuration conf, String restoreTmpDir, int expectedCount) throws IOException { FileSystem fs = FileSystem.get(conf); FileStatus[] subDirectories = fs.listStatus(new Path(restoreTmpDir)); @@ -429,67 +248,6 @@ private void checkRestoreTmpDir(Configuration conf, String restoreTmpDir, int ex } } - @Test - public void testVerifyReplicationWithSnapshotSupport() throws Exception { - // Populate the tables, at the same time it guarantees that the tables are - // identical since it does the check - runSmallBatchTest(); - - // Take source and target tables snapshot - Path rootDir = FSUtils.getRootDir(CONF1); - FileSystem fs = rootDir.getFileSystem(CONF1); - String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis(); - SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName, - Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true); - - // Take target snapshot - Path peerRootDir = FSUtils.getRootDir(CONF2); - FileSystem peerFs = peerRootDir.getFileSystem(CONF2); - String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis(); - SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName, - Bytes.toString(famName), peerSnapshotName, peerRootDir, peerFs, true); - - String peerFSAddress = peerFs.getUri().toString(); - String temPath1 = UTIL1.getRandomDir().toString(); - String temPath2 = "/tmp" + System.currentTimeMillis(); - - String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName, - "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName, - "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress, - "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() }; - runVerifyReplication(args, NB_ROWS_IN_BATCH, 0); - checkRestoreTmpDir(CONF1, temPath1, 1); - checkRestoreTmpDir(CONF2, temPath2, 1); - - Scan scan = new Scan(); - ResultScanner rs = htable2.getScanner(scan); - Put put = null; - for (Result result : rs) { - put = new Put(result.getRow()); - Cell firstVal = result.rawCells()[0]; - put.addColumn(CellUtil.cloneFamily(firstVal), CellUtil.cloneQualifier(firstVal), - Bytes.toBytes("diff data")); - htable2.put(put); - } - Delete delete = new Delete(put.getRow()); - htable2.delete(delete); - - sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis(); - SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName, - Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true); - - peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis(); - SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName, - Bytes.toString(famName), peerSnapshotName, peerRootDir, peerFs, true); - - args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName, - "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName, - "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress, - "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() }; - runVerifyReplication(args, 0, NB_ROWS_IN_BATCH); - checkRestoreTmpDir(CONF1, temPath1, 2); - checkRestoreTmpDir(CONF2, temPath2, 2); - } @Test public void testVerifyRepJobWithQuorumAddress() throws Exception { @@ -580,7 +338,7 @@ public void testVerifyRepJobWithQuorumAddressAndSnapshotSupport() throws Excepti checkRestoreTmpDir(CONF2, tmpPath2, 2); } - private static void runBatchCopyTest() throws Exception { + static void runBatchCopyTest() throws Exception { // normal Batch tests for htable1 loadData("", row, noRepfamName); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java new file mode 100644 index 000000000000..b3f4f6a1695e --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.replication; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +/** + * We moved some of {@link TestVerifyReplication}'s tests here because it could take too long to + * complete. In here we have miscellaneous. + */ +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestVerifyReplicationAdjunct extends TestReplicationBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestVerifyReplicationAdjunct.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestVerifyReplicationAdjunct.class); + + private static final String PEER_ID = "2"; + private static final TableName peerTableName = TableName.valueOf("peerTest"); + private static Table htable3; + + @Rule + public TestName name = new TestName(); + + @Before + public void setUp() throws Exception { + cleanUp(); + UTIL2.deleteTableData(peerTableName); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TestReplicationBase.setUpBeforeClass(); + TableDescriptor peerTable = TableDescriptorBuilder.newBuilder(peerTableName).setColumnFamily( + ColumnFamilyDescriptorBuilder.newBuilder(noRepfamName).setMaxVersions(100) + .build()).build(); + Connection connection2 = ConnectionFactory.createConnection(CONF2); + try (Admin admin2 = connection2.getAdmin()) { + admin2.createTable(peerTable, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); + } + htable3 = connection2.getTable(peerTableName); + } + + // VerifyReplication should honor versions option + @Test + public void testHBase14905() throws Exception { + // normal Batch tests + byte[] qualifierName = Bytes.toBytes("f1"); + Put put = new Put(Bytes.toBytes("r1")); + long ts = System.currentTimeMillis(); + put.addColumn(famName, qualifierName, ts + 1, Bytes.toBytes("v1002")); + htable1.put(put); + put.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v1001")); + htable1.put(put); + put.addColumn(famName, qualifierName, ts + 3, Bytes.toBytes("v1112")); + htable1.put(put); + + Scan scan = new Scan(); + scan.readVersions(100); + ResultScanner scanner1 = htable1.getScanner(scan); + Result[] res1 = scanner1.next(1); + scanner1.close(); + + assertEquals(1, res1.length); + assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); + + for (int i = 0; i < NB_RETRIES; i++) { + scan = new Scan(); + scan.readVersions(100); + scanner1 = htable2.getScanner(scan); + res1 = scanner1.next(1); + scanner1.close(); + if (res1.length != 1) { + LOG.info("Only got " + res1.length + " rows"); + Thread.sleep(SLEEP_TIME); + } else { + int cellNumber = res1[0].getColumnCells(famName, Bytes.toBytes("f1")).size(); + if (cellNumber != 3) { + LOG.info("Only got " + cellNumber + " cells"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + if (i == NB_RETRIES - 1) { + fail("Waited too much time for normal batch replication"); + } + } + + put.addColumn(famName, qualifierName, ts + 4, Bytes.toBytes("v1111")); + htable2.put(put); + put.addColumn(famName, qualifierName, ts + 5, Bytes.toBytes("v1112")); + htable2.put(put); + + scan = new Scan(); + scan.readVersions(100); + scanner1 = htable2.getScanner(scan); + res1 = scanner1.next(NB_ROWS_IN_BATCH); + scanner1.close(); + + assertEquals(1, res1.length); + assertEquals(5, res1[0].getColumnCells(famName, qualifierName).size()); + + String[] args = new String[] { "--versions=100", PEER_ID, tableName.getNameAsString() }; + TestVerifyReplication.runVerifyReplication(args, 0, 1); + } + + // VerifyReplication should honor versions option + @Test + public void testVersionMismatchHBase14905() throws Exception { + // normal Batch tests + byte[] qualifierName = Bytes.toBytes("f1"); + Put put = new Put(Bytes.toBytes("r1")); + long ts = System.currentTimeMillis(); + put.addColumn(famName, qualifierName, ts + 1, Bytes.toBytes("v1")); + htable1.put(put); + put.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v2")); + htable1.put(put); + put.addColumn(famName, qualifierName, ts + 3, Bytes.toBytes("v3")); + htable1.put(put); + + Scan scan = new Scan(); + scan.readVersions(100); + ResultScanner scanner1 = htable1.getScanner(scan); + Result[] res1 = scanner1.next(1); + scanner1.close(); + + assertEquals(1, res1.length); + assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); + + for (int i = 0; i < NB_RETRIES; i++) { + scan = new Scan(); + scan.readVersions(100); + scanner1 = htable2.getScanner(scan); + res1 = scanner1.next(1); + scanner1.close(); + if (res1.length != 1) { + LOG.info("Only got " + res1.length + " rows"); + Thread.sleep(SLEEP_TIME); + } else { + int cellNumber = res1[0].getColumnCells(famName, Bytes.toBytes("f1")).size(); + if (cellNumber != 3) { + LOG.info("Only got " + cellNumber + " cells"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + if (i == NB_RETRIES - 1) { + fail("Waited too much time for normal batch replication"); + } + } + + try { + // Disabling replication and modifying the particular version of the cell to validate the + // feature. + hbaseAdmin.disableReplicationPeer(PEER_ID); + Put put2 = new Put(Bytes.toBytes("r1")); + put2.addColumn(famName, qualifierName, ts + 2, Bytes.toBytes("v99")); + htable2.put(put2); + + scan = new Scan(); + scan.readVersions(100); + scanner1 = htable2.getScanner(scan); + res1 = scanner1.next(NB_ROWS_IN_BATCH); + scanner1.close(); + assertEquals(1, res1.length); + assertEquals(3, res1[0].getColumnCells(famName, qualifierName).size()); + + String[] args = new String[] { "--versions=100", PEER_ID, tableName.getNameAsString() }; + TestVerifyReplication.runVerifyReplication(args, 0, 1); + } finally { + hbaseAdmin.enableReplicationPeer(PEER_ID); + } + } + + @Test + public void testVerifyReplicationPrefixFiltering() throws Exception { + final byte[] prefixRow = Bytes.toBytes("prefixrow"); + final byte[] prefixRow2 = Bytes.toBytes("secondrow"); + loadData("prefixrow", prefixRow); + loadData("secondrow", prefixRow2); + loadData("aaa", row); + loadData("zzz", row); + waitForReplication(NB_ROWS_IN_BATCH * 4, NB_RETRIES * 4); + String[] args = + new String[] { "--row-prefixes=prefixrow,secondrow", PEER_ID, tableName.getNameAsString() }; + TestVerifyReplication.runVerifyReplication(args, NB_ROWS_IN_BATCH * 2, 0); + } + + @Test + public void testVerifyReplicationSnapshotArguments() { + String[] args = + new String[] { "--sourceSnapshotName=snapshot1", "2", tableName.getNameAsString() }; + assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--sourceSnapshotTmpDir=tmp", "2", tableName.getNameAsString() }; + assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--sourceSnapshotName=snapshot1", "--sourceSnapshotTmpDir=tmp", "2", + tableName.getNameAsString() }; + assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--peerSnapshotName=snapshot1", "2", tableName.getNameAsString() }; + assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--peerSnapshotTmpDir=/tmp/", "2", tableName.getNameAsString() }; + assertFalse(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--peerSnapshotName=snapshot1", "--peerSnapshotTmpDir=/tmp/", + "--peerFSAddress=tempfs", "--peerHBaseRootAddress=hdfs://tempfs:50070/hbase/", "2", + tableName.getNameAsString() }; + assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + + args = new String[] { "--sourceSnapshotName=snapshot1", "--sourceSnapshotTmpDir=/tmp/", + "--peerSnapshotName=snapshot2", "--peerSnapshotTmpDir=/tmp/", "--peerFSAddress=tempfs", + "--peerHBaseRootAddress=hdfs://tempfs:50070/hbase/", "2", tableName.getNameAsString() }; + + assertTrue(Lists.newArrayList(args).toString(), new VerifyReplication().doCommandLine(args)); + } + + @Test + public void testVerifyReplicationWithSnapshotSupport() throws Exception { + // Populate the tables, at the same time it guarantees that the tables are + // identical since it does the check + runSmallBatchTest(); + + // Take source and target tables snapshot + Path rootDir = FSUtils.getRootDir(CONF1); + FileSystem fs = rootDir.getFileSystem(CONF1); + String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis(); + SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName, + Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true); + + // Take target snapshot + Path peerRootDir = FSUtils.getRootDir(CONF2); + FileSystem peerFs = peerRootDir.getFileSystem(CONF2); + String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis(); + SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName, + Bytes.toString(famName), peerSnapshotName, peerRootDir, peerFs, true); + + String peerFSAddress = peerFs.getUri().toString(); + String temPath1 = UTIL1.getRandomDir().toString(); + String temPath2 = "/tmp" + System.currentTimeMillis(); + + String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName, + "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName, + "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress, + "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() }; + TestVerifyReplication.runVerifyReplication(args, NB_ROWS_IN_BATCH, 0); + TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 1); + TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 1); + + Scan scan = new Scan(); + ResultScanner rs = htable2.getScanner(scan); + Put put = null; + for (Result result : rs) { + put = new Put(result.getRow()); + Cell firstVal = result.rawCells()[0]; + put.addColumn(CellUtil.cloneFamily(firstVal), CellUtil.cloneQualifier(firstVal), + Bytes.toBytes("diff data")); + htable2.put(put); + } + Delete delete = new Delete(put.getRow()); + htable2.delete(delete); + + sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis(); + SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName, + Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true); + + peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis(); + SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName, + Bytes.toString(famName), peerSnapshotName, peerRootDir, peerFs, true); + + args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName, + "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName, + "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress, + "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() }; + TestVerifyReplication.runVerifyReplication(args, 0, NB_ROWS_IN_BATCH); + TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 2); + TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 2); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + htable3.close(); + TestReplicationBase.tearDownAfterClass(); + } +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java index 8b5b62a01345..a5469a4c3799 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java @@ -114,7 +114,7 @@ public void setUp() throws Exception { emptySnapshotName = "emptySnaptb0-" + testName.getMethodName(); // create Table - createTable(); + createTable(this.tableName); // Take an empty snapshot admin.snapshot(emptySnapshotName, tableName); @@ -127,7 +127,7 @@ public void setUp() throws Exception { admin.snapshot(snapshotName, tableName); } - protected void createTable() throws Exception { + protected void createTable(TableName tableName) throws Exception { SnapshotTestingUtils.createPreSplitTable(TEST_UTIL, tableName, 2, FAMILY); } @@ -171,7 +171,7 @@ public void testEmptyExportFileSystemState() throws Exception { @Test public void testConsecutiveExports() throws Exception { - Path copyDir = getLocalDestinationDir(); + Path copyDir = getLocalDestinationDir(TEST_UTIL); testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, false); testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, true); removeExportDir(copyDir); @@ -253,36 +253,6 @@ protected static void testExportFileSystemState(final Configuration conf, final assertEquals(filesExpected, snapshotFiles.size()); } - /** - * Check that ExportSnapshot will succeed if something fails but the retry succeed. - */ - @Test - public void testExportRetry() throws Exception { - Path copyDir = getLocalDestinationDir(); - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true); - conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 2); - conf.setInt("mapreduce.map.maxattempts", 3); - testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles, - TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), true); - } - - /** - * Check that ExportSnapshot will fail if we inject failure more times than MR will retry. - */ - @Test - public void testExportFailure() throws Exception { - Path copyDir = getLocalDestinationDir(); - FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); - copyDir = copyDir.makeQualified(fs); - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true); - conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 4); - conf.setInt("mapreduce.map.maxattempts", 3); - testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles, - TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), false); - } - /* * verify if the snapshot folder on file-system 1 match the one on file-system 2 */ @@ -291,12 +261,6 @@ protected static void verifySnapshotDir(final FileSystem fs1, final Path root1, assertEquals(listFiles(fs1, root1, root1), listFiles(fs2, root2, root2)); } - protected Set verifySnapshot(final FileSystem fs, final Path rootDir, - final TableName tableName, final String snapshotName) throws IOException { - return verifySnapshot(TEST_UTIL.getConfiguration(), fs, rootDir, tableName, - snapshotName, getBypassRegionPredicate()); - } - /* * Verify if the files exists */ @@ -364,10 +328,10 @@ private Path getHdfsDestinationDir() { return path; } - private Path getLocalDestinationDir() { - Path path = TEST_UTIL.getDataTestDir("local-export-" + System.currentTimeMillis()); + static Path getLocalDestinationDir(HBaseTestingUtility htu) { + Path path = htu.getDataTestDir("local-export-" + System.currentTimeMillis()); try { - FileSystem fs = FileSystem.getLocal(TEST_UTIL.getConfiguration()); + FileSystem fs = FileSystem.getLocal(htu.getConfiguration()); LOG.info("Local export destination path: " + path); return path.makeQualified(fs.getUri(), fs.getWorkingDirectory()); } catch (IOException ioe) { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotAdjunct.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotAdjunct.java new file mode 100644 index 000000000000..eed10115e6be --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotAdjunct.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.snapshot; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests that are adjunct to {@link TestExportSnapshot}. They used to be in same test suite but + * the test suite ran too close to the maximum time limit so we split these out. Uses + * facility from TestExportSnapshot where possible. + * @see TestExportSnapshot + */ +@Category({VerySlowMapReduceTests.class, LargeTests.class}) +public class TestExportSnapshotAdjunct { + private static final Logger LOG = LoggerFactory.getLogger(TestExportSnapshotAdjunct.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestExportSnapshotAdjunct.class); + @Rule + public final TestName testName = new TestName(); + + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + protected TableName tableName; + private String emptySnapshotName; + private String snapshotName; + private int tableNumFiles; + private Admin admin; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TestExportSnapshot.setUpBaseConf(TEST_UTIL.getConfiguration()); + TEST_UTIL.startMiniCluster(3); + TEST_UTIL.startMiniMapReduceCluster(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniMapReduceCluster(); + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Create a table and take a snapshot of the table used by the export test. + */ + @Before + public void setUp() throws Exception { + this.admin = TEST_UTIL.getAdmin(); + + tableName = TableName.valueOf("testtb-" + testName.getMethodName()); + snapshotName = "snaptb0-" + testName.getMethodName(); + emptySnapshotName = "emptySnaptb0-" + testName.getMethodName(); + + // Create Table + SnapshotTestingUtils.createPreSplitTable(TEST_UTIL, tableName, 2, TestExportSnapshot.FAMILY); + + // Take an empty snapshot + admin.snapshot(emptySnapshotName, tableName); + + // Add some rows + SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, + TestExportSnapshot.FAMILY); + tableNumFiles = admin.getRegions(tableName).size(); + + // take a snapshot + admin.snapshot(snapshotName, tableName); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.deleteTable(tableName); + SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getAdmin()); + SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); + } + + /** + * Check that ExportSnapshot will succeed if something fails but the retry succeed. + */ + @Test + public void testExportRetry() throws Exception { + Path copyDir = TestExportSnapshot.getLocalDestinationDir(TEST_UTIL); + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true); + conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 2); + conf.setInt("mapreduce.map.maxattempts", 3); + TestExportSnapshot.testExportFileSystemState(conf, tableName, + snapshotName, snapshotName, tableNumFiles, TEST_UTIL.getDefaultRootDirPath(), + copyDir, true, null, true); + } + + /** + * Check that ExportSnapshot will fail if we inject failure more times than MR will retry. + */ + @Test + public void testExportFailure() throws Exception { + Path copyDir = TestExportSnapshot.getLocalDestinationDir(TEST_UTIL); + FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); + copyDir = copyDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true); + conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 4); + conf.setInt("mapreduce.map.maxattempts", 3); + TestExportSnapshot.testExportFileSystemState(conf, tableName, + snapshotName, snapshotName, tableNumFiles, TEST_UTIL.getDefaultRootDirPath(), + copyDir, true, null, false); + } +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java index 59cdf4dc2130..27e5488a109a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; @@ -51,7 +52,7 @@ public static void setUpBeforeClass() throws Exception { } @Override - protected void createTable() throws Exception { + protected void createTable(TableName tableName) throws Exception { MobSnapshotTestingUtils.createPreSplitMobTable(TEST_UTIL, tableName, 2, FAMILY); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java index a805659fd646..b5c83d6f1b8d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java @@ -68,7 +68,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; import org.apache.hbase.thirdparty.io.netty.channel.ChannelId; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; -import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder; import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java index 48b762d75551..7b64075e5ded 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java @@ -99,7 +99,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; -import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder; import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java index 59215dea74df..090b9b4a63f1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java @@ -20,9 +20,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY; import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE; -import com.google.protobuf.ByteString; import com.google.protobuf.CodedOutputStream; import java.io.IOException; +import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -93,7 +93,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToByteEncoder; -import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder; import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent; import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler; @@ -355,15 +354,93 @@ private List getCipherOptions() throws IOException { return Collections.singletonList(new CipherOption(CipherSuite.AES_CTR_NOPADDING)); } + /** + * The asyncfs subsystem emulates a HDFS client by sending protobuf messages via netty. + * After Hadoop 3.3.0, the protobuf classes are relocated to org.apache.hadoop.thirdparty.protobuf.*. + * Use Reflection to check which ones to use. + */ + private static class BuilderPayloadSetter { + private static Method setPayloadMethod; + private static Constructor constructor; + + /** + * Create a ByteString from byte array without copying (wrap), and then set it as the payload + * for the builder. + * + * @param builder builder for HDFS DataTransferEncryptorMessage. + * @param payload byte array of payload. + * @throws IOException + */ + static void wrapAndSetPayload(DataTransferEncryptorMessageProto.Builder builder, byte[] payload) + throws IOException { + Object byteStringObject; + try { + // byteStringObject = new LiteralByteString(payload); + byteStringObject = constructor.newInstance(payload); + // builder.setPayload(byteStringObject); + setPayloadMethod.invoke(builder, constructor.getDeclaringClass().cast(byteStringObject)); + } catch (IllegalAccessException | InstantiationException e) { + throw new RuntimeException(e); + + } catch (InvocationTargetException e) { + Throwables.propagateIfPossible(e.getTargetException(), IOException.class); + throw new RuntimeException(e.getTargetException()); + } + } + + static { + Class builderClass = DataTransferEncryptorMessageProto.Builder.class; + + // Try the unrelocated ByteString + Class byteStringClass = com.google.protobuf.ByteString.class; + try { + // See if it can load the relocated ByteString, which comes from hadoop-thirdparty. + byteStringClass = Class.forName("org.apache.hadoop.thirdparty.protobuf.ByteString"); + LOG.debug("Found relocated ByteString class from hadoop-thirdparty." + + " Assuming this is Hadoop 3.3.0+."); + } catch (ClassNotFoundException e) { + LOG.debug("Did not find relocated ByteString class from hadoop-thirdparty." + + " Assuming this is below Hadoop 3.3.0", e); + } + + // LiteralByteString is a package private class in protobuf. Make it accessible. + Class literalByteStringClass; + try { + literalByteStringClass = Class.forName( + "org.apache.hadoop.thirdparty.protobuf.ByteString$LiteralByteString"); + LOG.debug("Shaded LiteralByteString from hadoop-thirdparty is found."); + } catch (ClassNotFoundException e) { + try { + literalByteStringClass = Class.forName("com.google.protobuf.LiteralByteString"); + LOG.debug("com.google.protobuf.LiteralByteString found."); + } catch (ClassNotFoundException ex) { + throw new RuntimeException(ex); + } + } + + try { + constructor = literalByteStringClass.getDeclaredConstructor(byte[].class); + constructor.setAccessible(true); + } catch (NoSuchMethodException e) { + throw new RuntimeException(e); + } + + try { + setPayloadMethod = builderClass.getMethod("setPayload", byteStringClass); + } catch (NoSuchMethodException e) { + // if either method is not found, we are in big trouble. Abort. + throw new RuntimeException(e); + } + } + } + private void sendSaslMessage(ChannelHandlerContext ctx, byte[] payload, List options) throws IOException { DataTransferEncryptorMessageProto.Builder builder = DataTransferEncryptorMessageProto.newBuilder(); builder.setStatus(DataTransferEncryptorStatus.SUCCESS); if (payload != null) { - // Was ByteStringer; fix w/o using ByteStringer. Its in hbase-protocol - // and we want to keep that out of hbase-server. - builder.setPayload(ByteString.copyFrom(payload)); + BuilderPayloadSetter.wrapAndSetPayload(builder, payload); } if (options != null) { builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/ProtobufDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/ProtobufDecoder.java new file mode 100644 index 000000000000..98b4e6f08e11 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/ProtobufDecoder.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.asyncfs; + +import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; +import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufUtil; +import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageDecoder; +import org.apache.hbase.thirdparty.io.netty.util.internal.ObjectUtil; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; + +/** + * Modified based on io.netty.handler.codec.protobuf.ProtobufDecoder. + * The Netty's ProtobufDecode supports unshaded protobuf messages (com.google.protobuf). + * + * Hadoop 3.3.0 and above relocates protobuf classes to a shaded jar (hadoop-thirdparty), and + * so we must use reflection to detect which one (relocated or not) to use. + * + * Do not use this to process HBase's shaded protobuf messages. This is meant to process the + * protobuf messages in HDFS for the asyncfs use case. + * */ +@InterfaceAudience.Private +public class ProtobufDecoder extends MessageToMessageDecoder { + private static final Logger LOG = + LoggerFactory.getLogger(ProtobufDecoder.class); + + private static Class protobufMessageLiteClass = null; + private static Class protobufMessageLiteBuilderClass = null; + + private static final boolean HAS_PARSER; + + private static Method getParserForTypeMethod; + private static Method newBuilderForTypeMethod; + + private Method parseFromMethod; + private Method mergeFromMethod; + private Method buildMethod; + + private Object parser; + private Object builder; + + + public ProtobufDecoder(Object prototype) { + try { + Method getDefaultInstanceForTypeMethod = protobufMessageLiteClass.getMethod( + "getDefaultInstanceForType"); + Object prototype1 = getDefaultInstanceForTypeMethod + .invoke(ObjectUtil.checkNotNull(prototype, "prototype")); + + // parser = prototype.getParserForType() + parser = getParserForTypeMethod.invoke(prototype1); + parseFromMethod = parser.getClass().getMethod( + "parseFrom", byte[].class, int.class, int.class); + + // builder = prototype.newBuilderForType(); + builder = newBuilderForTypeMethod.invoke(prototype1); + mergeFromMethod = builder.getClass().getMethod( + "mergeFrom", byte[].class, int.class, int.class); + + // All protobuf message builders inherits from MessageLite.Builder + buildMethod = protobufMessageLiteBuilderClass.getDeclaredMethod("build"); + + } catch (IllegalAccessException | NoSuchMethodException e) { + throw new RuntimeException(e); + } catch (InvocationTargetException e) { + throw new RuntimeException(e.getTargetException()); + } + } + + protected void decode( + ChannelHandlerContext ctx, ByteBuf msg, List out) throws Exception { + int length = msg.readableBytes(); + byte[] array; + int offset; + if (msg.hasArray()) { + array = msg.array(); + offset = msg.arrayOffset() + msg.readerIndex(); + } else { + array = ByteBufUtil.getBytes(msg, msg.readerIndex(), length, false); + offset = 0; + } + + Object addObj; + if (HAS_PARSER) { + // addObj = parser.parseFrom(array, offset, length); + addObj = parseFromMethod.invoke(parser, array, offset, length); + } else { + // addObj = builder.mergeFrom(array, offset, length).build(); + Object builderObj = mergeFromMethod.invoke(builder, array, offset, length); + addObj = buildMethod.invoke(builderObj); + } + out.add(addObj); + } + + static { + boolean hasParser = false; + + // These are the protobuf classes coming from Hadoop. Not the one from hbase-shaded-protobuf + protobufMessageLiteClass = com.google.protobuf.MessageLite.class; + protobufMessageLiteBuilderClass = com.google.protobuf.MessageLite.Builder.class; + + try { + protobufMessageLiteClass = Class.forName("org.apache.hadoop.thirdparty.protobuf.MessageLite"); + protobufMessageLiteBuilderClass = Class.forName( + "org.apache.hadoop.thirdparty.protobuf.MessageLite.Builder"); + LOG.debug("Hadoop 3.3 and above shades protobuf."); + } catch (ClassNotFoundException e) { + LOG.debug("Hadoop 3.2 and below use unshaded protobuf.", e); + } + + try { + getParserForTypeMethod = protobufMessageLiteClass.getDeclaredMethod("getParserForType"); + newBuilderForTypeMethod = protobufMessageLiteClass.getDeclaredMethod("newBuilderForType"); + } catch (NoSuchMethodException e) { + // If the method is not found, we are in trouble. Abort. + throw new RuntimeException(e); + } + + try { + protobufMessageLiteClass.getDeclaredMethod("getParserForType"); + hasParser = true; + } catch (Throwable var2) { + } + + HAS_PARSER = hasParser; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 88f73f79aae2..a8d1d1d26d4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TimeZone; @@ -437,17 +438,16 @@ private void scanKeysValues(Path file, KeyValueStatsCollector fileStats, } // check if mob files are missing. if (checkMobIntegrity && MobUtils.isMobReferenceCell(cell)) { - Tag tnTag = MobUtils.getTableNameTag(cell); - if (tnTag == null) { + Optional tn = MobUtils.getTableName(cell); + if (! tn.isPresent()) { System.err.println("ERROR, wrong tag format in mob reference cell " + CellUtil.getCellKeyAsString(cell)); } else if (!MobUtils.hasValidMobRefCellValue(cell)) { System.err.println("ERROR, wrong value format in mob reference cell " + CellUtil.getCellKeyAsString(cell)); } else { - TableName tn = TableName.valueOf(Tag.cloneValue(tnTag)); String mobFileName = MobUtils.getMobFileName(cell); - boolean exist = mobFileExists(fs, tn, mobFileName, + boolean exist = mobFileExists(fs, tn.get(), mobFileName, Bytes.toString(CellUtil.cloneFamily(cell)), foundMobFiles, missingMobFiles); if (!exist) { // report error diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index ce7a583e2974..6b993fe295f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.regionserver.slowlog.RpcLogDetails; import org.apache.hadoop.hbase.regionserver.slowlog.SlowLogRecorder; +import org.apache.hadoop.hbase.security.HBasePolicyProvider; import org.apache.hadoop.hbase.security.SaslUtil; import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; import org.apache.hadoop.hbase.security.User; @@ -58,6 +59,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.authorize.PolicyProvider; +import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.authorize.ServiceAuthorizationManager; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; @@ -313,6 +315,14 @@ public void onConfigurationChange(Configuration newConf) { if (scheduler instanceof ConfigurationObserver) { ((ConfigurationObserver) scheduler).onConfigurationChange(newConf); } + // Make sure authManager will read hbase-policy file + System.setProperty("hadoop.policy.file", "hbase-policy.xml"); + synchronized (authManager) { + authManager.refresh(newConf, new HBasePolicyProvider()); + } + LOG.info("Refreshed hbase-policy.xml successfully"); + ProxyUsers.refreshSuperUserGroupsConfiguration(newConf); + LOG.info("Refreshed super and proxy users successfully"); } protected void initReconfigurable(Configuration confToLoad) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java index 3675bd3d07f2..0698d128eb30 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java @@ -65,4 +65,9 @@ public boolean equals(Object o) { } return false; } + + @Override + public String toString() { + return "server=" + sn + " , load=" + load; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java index c9fe0908335a..e63a6b906612 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java @@ -331,28 +331,29 @@ protected boolean needsBalance(TableName tableName, Cluster cluster) { for (CostFunction c : costFunctions) { float multiplier = c.getMultiplier(); if (multiplier <= 0) { + LOG.trace("{} not needed because multiplier is <= 0", c.getClass().getSimpleName()); continue; } if (!c.isNeeded()) { - LOG.debug("{} not needed", c.getClass().getSimpleName()); + LOG.trace("{} not needed", c.getClass().getSimpleName()); continue; } sumMultiplier += multiplier; total += c.cost() * multiplier; } - if (total <= 0 || sumMultiplier <= 0 - || (sumMultiplier > 0 && (total / sumMultiplier) < minCostNeedBalance)) { + boolean balanced = total <= 0 || sumMultiplier <= 0 || + (sumMultiplier > 0 && (total / sumMultiplier) < minCostNeedBalance); + if (LOG.isDebugEnabled()) { + LOG.debug("{} {}; total cost={}, sum multiplier={}; cost/multiplier to need a balance is {}", + balanced ? "Skipping load balancing because balanced" : "We need to load balance", + isByTable ? String.format("table (%s)", tableName) : "cluster", + total, sumMultiplier, minCostNeedBalance); if (LOG.isTraceEnabled()) { - final String loadBalanceTarget = - isByTable ? String.format("table (%s)", tableName) : "cluster"; - LOG.trace("Skipping load balancing because the {} is balanced. Total cost: {}, " - + "Sum multiplier: {}, Minimum cost needed for balance: {}", loadBalanceTarget, total, - sumMultiplier, minCostNeedBalance); + LOG.trace("Balance decision detailed function costs={}", functionCost()); } - return false; } - return true; + return !balanced; } @VisibleForTesting @@ -1189,16 +1190,27 @@ static class RegionCountSkewCostFunction extends CostFunction { this.setMultiplier(conf.getFloat(REGION_COUNT_SKEW_COST_KEY, DEFAULT_REGION_COUNT_SKEW_COST)); } + @Override + void init(Cluster cluster) { + super.init(cluster); + LOG.debug("{} sees a total of {} servers and {} regions.", getClass().getSimpleName(), + cluster.numServers, cluster.numRegions); + if (LOG.isTraceEnabled()) { + for (int i =0; i < cluster.numServers; i++) { + LOG.trace("{} sees server '{}' has {} regions", getClass().getSimpleName(), + cluster.servers[i], cluster.regionsPerServer[i].length); + } + } + } + @Override protected double cost() { if (stats == null || stats.length != cluster.numServers) { stats = new double[cluster.numServers]; } - for (int i =0; i < cluster.numServers; i++) { stats[i] = cluster.regionsPerServer[i].length; } - return costFromArray(stats); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index c5ef6a6b992b..45b9d80573dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -26,9 +26,9 @@ import java.util.ArrayList; import java.util.Date; import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Set; +import java.util.Map.Entry; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -37,9 +37,11 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -60,7 +62,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSetMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap; /** * Compact passed set of files in the mob-enabled column family. @@ -79,13 +84,8 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { * content of it is written into meta section of a newly created store file at the final step of * compaction process. */ - - static ThreadLocal> mobRefSet = new ThreadLocal>() { - @Override - protected Set initialValue() { - return new HashSet(); - } - }; + static ThreadLocal> mobRefSet = + ThreadLocal.withInitial(HashMultimap::create); /* * Is it user or system-originated request. @@ -190,34 +190,71 @@ public List compact(CompactionRequestImpl request, // Check if I/O optimized MOB compaction if (ioOptimizedMode) { if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { - Path mobDir = - MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); - List mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir); - //reset disableIO - disableIO.set(Boolean.FALSE); - if (mobFiles.size() > 0) { - calculateMobLengthMap(mobFiles); + try { + final SetMultimap mobRefs = request.getFiles().stream() + .map(file -> { + byte[] value = file.getMetadataValue(HStoreFile.MOB_FILE_REFS); + ImmutableSetMultimap.Builder builder; + if (value == null) { + builder = ImmutableSetMultimap.builder(); + } else { + try { + builder = MobUtils.deserializeMobFileRefs(value); + } catch (RuntimeException exception) { + throw new RuntimeException("failure getting mob references for hfile " + file, + exception); + } + } + return builder; + }).reduce((a, b) -> a.putAll(b.build())).orElseGet(ImmutableSetMultimap::builder) + .build(); + //reset disableIO + disableIO.set(Boolean.FALSE); + if (!mobRefs.isEmpty()) { + calculateMobLengthMap(mobRefs); + } + LOG.info("Table={} cf={} region={}. I/O optimized MOB compaction. "+ + "Total referenced MOB files: {}", tableName, familyName, regionName, mobRefs.size()); + } catch (RuntimeException exception) { + throw new IOException("Failed to get list of referenced hfiles for request " + request, + exception); } - LOG.info("Table={} cf={} region={}. I/O optimized MOB compaction. "+ - "Total referenced MOB files: {}", tableName, familyName, regionName, mobFiles.size()); } } return compact(request, scannerFactory, writerFactory, throughputController, user); } - private void calculateMobLengthMap(List mobFiles) throws IOException { + /** + * @param mobRefs multimap of original table name -> mob hfile + */ + private void calculateMobLengthMap(SetMultimap mobRefs) throws IOException { FileSystem fs = store.getFileSystem(); HashMap map = mobLengthMap.get(); map.clear(); - for (Path p : mobFiles) { - if (MobFileName.isOldMobFileName(p.getName())) { + for (Entry reference : mobRefs.entries()) { + final TableName table = reference.getKey(); + final String mobfile = reference.getValue(); + if (MobFileName.isOldMobFileName(mobfile)) { disableIO.set(Boolean.TRUE); } - FileStatus st = fs.getFileStatus(p); - long size = st.getLen(); - LOG.debug("Referenced MOB file={} size={}", p, size); - map.put(p.getName(), fs.getFileStatus(p).getLen()); + List locations = mobStore.getLocations(table); + for (Path p : locations) { + try { + FileStatus st = fs.getFileStatus(new Path(p, mobfile)); + long size = st.getLen(); + LOG.debug("Referenced MOB file={} size={}", mobfile, size); + map.put(mobfile, size); + break; + } catch (FileNotFoundException exception) { + LOG.debug("Mob file {} was not in location {}. May have other locations to try.", mobfile, + p); + } + } + if (!map.containsKey(mobfile)) { + throw new FileNotFoundException("Could not find mob file " + mobfile + " in the list of " + + "expected locations: " + locations); + } } } @@ -391,8 +428,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // We leave large MOB file as is (is not compacted), // then we update set of MOB file references // and append mob cell directly to the store's writer - mobRefSet.get().add(fName); - writer.append(mobCell); + Optional refTable = MobUtils.getTableName(c); + if (refTable.isPresent()) { + mobRefSet.get().put(refTable.get(), fName); + writer.append(c); + } else { + throw new IOException(String.format("MOB cell did not contain a tablename " + + "tag. should not be possible. see ref guide on mob troubleshooting. " + + "store={} cell={}", getStoreInfo(), c)); + } } } } else { @@ -440,9 +484,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (MobUtils.hasValidMobRefCellValue(c)) { // We do not check mobSizeThreshold during normal compaction, // leaving it to a MOB compaction run - writer.append(c); - // Add MOB reference to a MOB reference set - mobRefSet.get().add(MobUtils.getMobFileName(c)); + Optional refTable = MobUtils.getTableName(c); + if (refTable.isPresent()) { + mobRefSet.get().put(refTable.get(), MobUtils.getMobFileName(c)); + writer.append(c); + } else { + throw new IOException(String.format("MOB cell did not contain a tablename " + + "tag. should not be possible. see ref guide on mob troubleshooting. " + + "store={} cell={}", getStoreInfo(), c)); + } } else { String errMsg = String.format("Corrupted MOB reference: %s", c.toString()); throw new IOException(errMsg); @@ -529,7 +579,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { // Remove all MOB references because compaction failed - mobRefSet.get().clear(); + clearThreadLocals(); // Abort writer LOG.debug("Aborting writer for {} because of a compaction failure, Store {}", mobFileWriter.getPath(), getStoreInfo()); @@ -547,16 +597,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel return true; } - private String getStoreInfo() { + protected String getStoreInfo() { return String.format("[table=%s family=%s region=%s]", store.getTableName().getNameAsString(), store.getColumnFamilyName(), store.getRegionInfo().getEncodedName()) ; } private void clearThreadLocals() { - Set set = mobRefSet.get(); - if (set != null) { - set.clear(); - } + mobRefSet.get().clear(); HashMap map = mobLengthMap.get(); if (map != null) { map.clear(); @@ -571,7 +618,7 @@ private StoreFileWriter newMobWriter(FileDetails fd) LOG.debug("New MOB writer created={} store={}", mobFileWriter.getPath().getName(), getStoreInfo()); // Add reference we get for compact MOB - mobRefSet.get().add(mobFileWriter.getPath().getName()); + mobRefSet.get().put(store.getTableName(), mobFileWriter.getPath().getName()); return mobFileWriter; } catch (IOException e) { // Bailing out @@ -604,7 +651,7 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId LOG.debug("Aborting writer for {} because there are no MOB cells, store={}", mobFileWriter.getPath(), getStoreInfo()); // Remove MOB file from reference set - mobRefSet.get().remove(mobFileWriter.getPath().getName()); + mobRefSet.get().remove(store.getTableName(), mobFileWriter.getPath().getName()); abortWriter(mobFileWriter); } } else { @@ -619,9 +666,7 @@ protected List commitWriter(StoreFileWriter writer, FileDetails fd, CompactionRequestImpl request) throws IOException { List newFiles = Lists.newArrayList(writer.getPath()); writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); - // Append MOB references - Set refSet = mobRefSet.get(); - writer.appendMobMetadata(refSet); + writer.appendMobMetadata(mobRefSet.get()); writer.close(); clearThreadLocals(); return newFiles; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 5c4c6020ae09..c1d573601f83 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; @@ -48,6 +49,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSetMultimap; + /** * An implementation of the StoreFlusher. It extends the DefaultStoreFlusher. * If the store is not a mob store, the flusher flushes the MemStore the same with @@ -280,7 +283,8 @@ protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, // The hfile is current up to and including cacheFlushSeqNum. status.setStatus("Flushing " + store + ": appending metadata"); writer.appendMetadata(cacheFlushSeqNum, false); - writer.appendMobMetadata(mobRefSet.get()); + writer.appendMobMetadata(ImmutableSetMultimap.builder() + .putAll(store.getTableName(), mobRefSet.get()).build()); status.setStatus("Flushing " + store + ": closing flushed file"); writer.close(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java index 7f7d90c5f64b..8ea25b7b5d21 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java @@ -21,7 +21,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -54,6 +53,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap; /** * The class MobFileCleanerChore for running cleaner regularly to remove the expired @@ -212,28 +212,28 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); // close store file to avoid memory leaks sf.closeStoreFile(true); - if (mobRefData == null && bulkloadMarkerData == null) { - LOG.warn("Found old store file with no MOB_FILE_REFS: {} - " - + "can not proceed until all old files will be MOB-compacted.", - pp); - return; - } else if (mobRefData == null && bulkloadMarkerData != null) { - LOG.debug("Skipping file without MOB references (bulkloaded file):{}", pp); - continue; - } - // mobRefData will never be null here, but to make FindBugs happy - if (mobRefData != null && mobRefData.length > 1) { - // if length = 1 means NULL, that there are no MOB references - // in this store file, but the file was created by new MOB code - String[] mobs = new String(mobRefData).split(","); - if (LOG.isTraceEnabled()) { - LOG.trace("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs)); + if (mobRefData == null) { + if (bulkloadMarkerData == null) { + LOG.warn("Found old store file with no MOB_FILE_REFS: {} - " + + "can not proceed until all old files will be MOB-compacted.", + pp); + return; } else { - LOG.debug("Found: {} mob references", mobs.length); + LOG.debug("Skipping file without MOB references (bulkloaded file):{}", pp); + continue; } - regionMobs.addAll(Arrays.asList(mobs)); - } else { - LOG.debug("File {} does not have mob references", currentPath); + } + // file may or may not have MOB references, but was created by the distributed + // mob compaction code. + try { + SetMultimap mobs = MobUtils.deserializeMobFileRefs(mobRefData) + .build(); + LOG.debug("Found {} mob references for store={}", mobs.size(), sf); + LOG.trace("Specific mob references found for store={} : {}", sf, mobs); + regionMobs.addAll(mobs.values()); + } catch (RuntimeException exception) { + throw new IOException("failure getting mob references for hfile " + sf, + exception); } } } catch (FileNotFoundException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 3a6a55da9e34..8ce63fa30c0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -20,17 +20,16 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.ByteBuffer; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; import java.util.Calendar; import java.util.Collection; -import java.util.Collections; import java.util.Date; -import java.util.HashSet; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.UUID; import org.apache.hadoop.conf.Configuration; @@ -68,6 +67,8 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSetMultimap; +import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap; /** * The mob utilities @@ -130,14 +131,51 @@ public static boolean isMobReferenceCell(Cell cell) { * @param cell The current cell. * @return The table name tag. */ - public static Tag getTableNameTag(Cell cell) { + private static Optional getTableNameTag(Cell cell) { + Optional tag = Optional.empty(); if (cell.getTagsLength() > 0) { - Optional tag = PrivateCellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE); - if (tag.isPresent()) { - return tag.get(); + tag = PrivateCellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE); + } + return tag; + } + + /** + * Gets the table name from when this cell was written into a mob hfile as a string. + * @param cell to extract tag from + * @return table name as a string. empty if the tag is not found. + */ + public static Optional getTableNameString(Cell cell) { + Optional tag = getTableNameTag(cell); + Optional name = Optional.empty(); + if (tag.isPresent()) { + name = Optional.of(Tag.getValueAsString(tag.get())); + } + return name; + } + + /** + * Get the table name from when this cell was written into a mob hfile as a TableName. + * @param cell to extract tag from + * @return name of table as a TableName. empty if the tag is not found. + */ + public static Optional getTableName(Cell cell) { + Optional maybe = getTableNameTag(cell); + Optional name = Optional.empty(); + if (maybe.isPresent()) { + final Tag tag = maybe.get(); + if (tag.hasArray()) { + name = Optional.of(TableName.valueOf(tag.getValueArray(), tag.getValueOffset(), + tag.getValueLength())); + } else { + // TODO ByteBuffer handling in tags looks busted. revisit. + ByteBuffer buffer = tag.getValueByteBuffer().duplicate(); + buffer.mark(); + buffer.position(tag.getValueOffset()); + buffer.limit(tag.getValueOffset() + tag.getValueLength()); + name = Optional.of(TableName.valueOf(buffer)); } } - return null; + return name; } /** @@ -383,7 +421,7 @@ public static Path getMobFamilyPath(Path regionPath, String familyName) { /** * Gets the RegionInfo of the mob files. This is a dummy region. The mob files are not saved in a - * region in HBase. This is only used in mob snapshot. It's internally used only. + * region in HBase. It's internally used only. * @param tableName * @return A dummy mob region info. */ @@ -665,27 +703,78 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre } /** - * Get list of referenced MOB files from a given collection of store files - * @param storeFiles store files - * @param mobDir MOB file directory - * @return list of MOB file paths - */ - - public static List getReferencedMobFiles(Collection storeFiles, Path mobDir) { - - Set mobSet = new HashSet(); - for (HStoreFile sf : storeFiles) { - byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - if (value != null && value.length > 1) { - String s = Bytes.toString(value); - String[] all = s.split(","); - Collections.addAll(mobSet, all); + * Serialize a set of referenced mob hfiles + * @param mobRefSet to serialize, may be null + * @return byte array to i.e. put into store file metadata. will not be null + */ + public static byte[] serializeMobFileRefs(SetMultimap mobRefSet) { + if (mobRefSet != null && mobRefSet.size() > 0) { + // Here we rely on the fact that '/' and ',' are not allowed in either table names nor hfile + // names for serialization. + // + // exampleTable/filename1,filename2//example:table/filename5//otherTable/filename3,filename4 + // + // to approximate the needed capacity we use the fact that there will usually be 1 table name + // and each mob filename is around 105 bytes. we pick an arbitrary number to cover "most" + // single table name lengths + StringBuilder sb = new StringBuilder(100 + mobRefSet.size() * 105); + boolean doubleSlash = false; + for (TableName tableName : mobRefSet.keySet()) { + sb.append(tableName).append("/"); + boolean comma = false; + for (String refs : mobRefSet.get(tableName)) { + sb.append(refs); + if (comma) { + sb.append(","); + } else { + comma = true; + } + } + if (doubleSlash) { + sb.append("//"); + } else { + doubleSlash = true; + } } + return Bytes.toBytes(sb.toString()); + } else { + return HStoreFile.NULL_VALUE; } - List retList = new ArrayList(); - for (String name : mobSet) { - retList.add(new Path(mobDir, name)); + } + + /** + * Deserialize the set of referenced mob hfiles from store file metadata. + * @param bytes compatibly serialized data. can not be null + * @return a setmultimap of original table to list of hfile names. will be empty if no values. + * @throws IllegalStateException if there are values but no table name + */ + public static ImmutableSetMultimap.Builder deserializeMobFileRefs(byte[] bytes) + throws IllegalStateException { + ImmutableSetMultimap.Builder map = ImmutableSetMultimap.builder(); + if (bytes.length > 1) { + // TODO avoid turning the tablename pieces in to strings. + String s = Bytes.toString(bytes); + String[] tables = s.split("//"); + for (String tableEnc : tables) { + final int delim = tableEnc.indexOf('/'); + if (delim <= 0) { + throw new IllegalStateException("MOB reference data does not match expected encoding: " + + "no table name included before list of mob refs."); + } + TableName table = TableName.valueOf(tableEnc.substring(0, delim)); + String[] refs = tableEnc.substring(delim + 1).split(","); + map.putAll(table, refs); + } + } else { + if (LOG.isDebugEnabled()) { + // array length 1 should be the NULL_VALUE. + if (! Arrays.equals(HStoreFile.NULL_VALUE, bytes)) { + LOG.debug("Serialized MOB file refs array was treated as the placeholder 'no entries' but" + + " didn't have the expected placeholder byte. expected={} and actual={}", + Arrays.toString(HStoreFile.NULL_VALUE), Arrays.toString(bytes)); + } + } } - return retList; + return map; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index d7f2ba3999f8..5960b8030900 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -55,7 +56,6 @@ import org.apache.hadoop.hbase.mob.MobFileName; import org.apache.hadoop.hbase.mob.MobStoreEngine; import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.IdLock; import org.apache.yetus.audience.InterfaceAudience; @@ -94,7 +94,7 @@ public class HMobStore extends HStore { private AtomicLong mobScanCellsCount = new AtomicLong(); private AtomicLong mobScanCellsSize = new AtomicLong(); private ColumnFamilyDescriptor family; - private Map> map = new ConcurrentHashMap<>(); + private Map> map = new ConcurrentHashMap<>(); private final IdLock keyLock = new IdLock(); // When we add a MOB reference cell to the HFile, we will add 2 tags along with it // 1. A ref tag with type TagType.MOB_REFERENCE_TAG_TYPE. This just denote this this cell is not @@ -117,7 +117,7 @@ public HMobStore(final HRegion region, final ColumnFamilyDescriptor family, TableName tn = region.getTableDescriptor().getTableName(); locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn) .getEncodedName(), family.getNameAsString())); - map.put(Bytes.toString(tn.getName()), locations); + map.put(tn, locations); List tags = new ArrayList<>(2); tags.add(MobConstants.MOB_REF_TAG); Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, @@ -315,26 +315,9 @@ public MobCell resolve(Cell reference, boolean cacheBlocks, long readPt, MobCell mobCell = null; if (MobUtils.hasValidMobRefCellValue(reference)) { String fileName = MobUtils.getMobFileName(reference); - Tag tableNameTag = MobUtils.getTableNameTag(reference); - if (tableNameTag != null) { - String tableNameString = Tag.getValueAsString(tableNameTag); - List locations = map.get(tableNameString); - if (locations == null) { - IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode()); - try { - locations = map.get(tableNameString); - if (locations == null) { - locations = new ArrayList<>(2); - TableName tn = TableName.valueOf(tableNameString); - locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString())); - locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, - MobUtils.getMobRegionInfo(tn).getEncodedName(), family.getNameAsString())); - map.put(tableNameString, locations); - } - } finally { - keyLock.releaseLockEntry(lockEntry); - } - } + Optional tableName = MobUtils.getTableName(reference); + if (tableName.isPresent()) { + List locations = getLocations(tableName.get()); mobCell = readCell(locations, fileName, reference, cacheBlocks, readPt, readEmptyValueOnMobCellMiss); } @@ -357,6 +340,30 @@ public MobCell resolve(Cell reference, boolean cacheBlocks, long readPt, return mobCell; } + /** + * @param tableName to look up locations for, can not be null + * @return a list of location in order of working dir, archive dir. will not be null. + */ + public List getLocations(TableName tableName) throws IOException { + List locations = map.get(tableName); + if (locations == null) { + IdLock.Entry lockEntry = keyLock.getLockEntry(tableName.hashCode()); + try { + locations = map.get(tableName); + if (locations == null) { + locations = new ArrayList<>(2); + locations.add(MobUtils.getMobFamilyPath(conf, tableName, family.getNameAsString())); + locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tableName, + MobUtils.getMobRegionInfo(tableName).getEncodedName(), family.getNameAsString())); + map.put(tableName, locations); + } + } finally { + keyLock.releaseLockEntry(lockEntry); + } + } + return locations; + } + /** * Reads the cell from a mob file. * The mob file might be located in different directories. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 3a12f87b9fe6..35718cc88493 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -681,7 +681,7 @@ void sawNoSuchFamily() { private long flushPerChanges; private long blockingMemStoreSize; // Used to guard closes - final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + final ReentrantReadWriteLock lock; // Stop updates lock private final ReentrantReadWriteLock updatesLock = new ReentrantReadWriteLock(); @@ -772,6 +772,8 @@ public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration co this.conf = new CompoundConfiguration() .add(confParam) .addBytesMap(htd.getValues()); + this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK, + DEFAULT_FAIR_REENTRANT_CLOSE_LOCK)); this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL, DEFAULT_CACHE_FLUSH_INTERVAL); this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES); @@ -1521,6 +1523,10 @@ public Map> close() throws IOException { private final Object closeLock = new Object(); + /** Conf key for fair locking policy */ + public static final String FAIR_REENTRANT_CLOSE_LOCK = + "hbase.regionserver.fair.region.close.lock"; + public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true; /** Conf key for the periodic flush interval */ public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL = "hbase.regionserver.optionalcacheflushinterval"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 0f227be472fc..0de0295d6d24 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -297,8 +297,7 @@ public long getModificationTimestamp() throws IOException { } /** - * Only used by the Striped Compaction Policy - * @param key + * @param key to look up * @return value associated with the metadata key */ public byte[] getMetadataValue(byte[] key) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 3de97e80e195..10aa267f34d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -26,7 +26,6 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; import java.net.InetSocketAddress; @@ -39,7 +38,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -47,11 +45,13 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; +import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.BloomContext; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterUtil; @@ -65,6 +65,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** @@ -248,17 +249,11 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti /** * Appends MOB - specific metadata (even if it is empty) - * @param mobRefSet - set of MOB file names + * @param mobRefSet - original table -> set of MOB file names * @throws IOException problem writing to FS */ - public void appendMobMetadata(Set mobRefSet) throws IOException { - if (mobRefSet != null && mobRefSet.size() > 0) { - String sb = StringUtils.join(mobRefSet, ","); - byte[] bytes = Bytes.toBytes(sb.toString()); - writer.appendFileInfo(MOB_FILE_REFS, bytes); - } else { - writer.appendFileInfo(MOB_FILE_REFS, NULL_VALUE); - } + public void appendMobMetadata(SetMultimap mobRefSet) throws IOException { + writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet)); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java index 96be61c99b4d..c48caad7c7ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java @@ -445,7 +445,7 @@ private IOException extractHiddenEof(Exception ex) { && ex.getCause() != null && ex.getCause() instanceof IOException) { ioEx = (IOException)ex.getCause(); } - if (ioEx != null) { + if ((ioEx != null) && (ioEx.getMessage() != null)) { if (ioEx.getMessage().contains("EOF")) return ioEx; return null; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java new file mode 100644 index 000000000000..81ec0d9129cd --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.replication.regionserver; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This exception is thrown when a replication source is terminated and source threads got + * interrupted. + * + * It is inherited from RuntimeException so that it can skip all the following processing logic + * and be propagated to the most top level and handled there. + */ +@InterfaceAudience.Private +public class ReplicationRuntimeException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public ReplicationRuntimeException(String m, Throwable t) { + super(m, t); + } + + public ReplicationRuntimeException(String m) { + super(m); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index 43afa798f08b..1a012bd5db42 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -496,11 +496,13 @@ public void refreshSources(String peerId) throws IOException { // synchronized on oldsources to avoid race with NodeFailoverWorker synchronized (this.oldsources) { List previousQueueIds = new ArrayList<>(); - for (ReplicationSourceInterface oldSource : this.oldsources) { + for (Iterator iter = this.oldsources.iterator(); iter + .hasNext();) { + ReplicationSourceInterface oldSource = iter.next(); if (oldSource.getPeerId().equals(peerId)) { previousQueueIds.add(oldSource.getQueueId()); oldSource.terminate(terminateMessage); - this.oldsources.remove(oldSource); + iter.remove(); } } for (String queueId : previousQueueIds) { @@ -579,8 +581,13 @@ private void interruptOrAbortWhenFail(ReplicationQueueOperation op) { if (e.getCause() != null && e.getCause() instanceof KeeperException.SystemErrorException && e.getCause().getCause() != null && e.getCause() .getCause() instanceof InterruptedException) { - throw new RuntimeException( - "Thread is interrupted, the replication source may be terminated"); + // ReplicationRuntimeException(a RuntimeException) is thrown out here. The reason is + // that thread is interrupted deep down in the stack, it should pass the following + // processing logic and propagate to the most top layer which can handle this exception + // properly. In this specific case, the top layer is ReplicationSourceShipper#run(). + throw new ReplicationRuntimeException( + "Thread is interrupted, the replication source may be terminated", + e.getCause().getCause()); } server.abort("Failed to operate on replication queue", e); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java index cf42d4ed0260..732b6872c0bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java @@ -119,8 +119,9 @@ public final void run() { } else { shipEdits(entryBatch); } - } catch (InterruptedException e) { - LOG.trace("Interrupted while waiting for next replication entry batch", e); + } catch (InterruptedException | ReplicationRuntimeException e) { + // It is interrupted and needs to quit. + LOG.warn("Interrupted while waiting for next replication entry batch", e); Thread.currentThread().interrupt(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java index 39e70adb7f77..5a2a8939826c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java @@ -175,7 +175,10 @@ public void testEntityLockTimeout() throws Exception { assertTrue(lock.isLocked()); // Should get unlocked in next heartbeat i.e. after workerSleepTime. Wait 10x time to be sure. assertTrue(waitLockTimeOut(lock, 10 * workerSleepTime)); - assertFalse(lock.getWorker().isAlive()); + + // Works' run() returns, there is a small gap that the thread is still alive(os + // has not declare it is dead yet), so remove the following assertion. + // assertFalse(lock.getWorker().isAlive()); verify(abortable, times(1)).abort(any(), eq(null)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 623105968c7c..685088fed2c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -201,9 +201,11 @@ public void assertClusterAsBalanced(List servers) { int max = numRegions % numServers == 0 ? min : min + 1; for (ServerAndLoad server : servers) { - assertTrue(server.getLoad() >= 0); - assertTrue(server.getLoad() <= max); - assertTrue(server.getLoad() >= min); + assertTrue("All servers should have a positive load. " + server, server.getLoad() >= 0); + assertTrue("All servers should have load no more than " + max + ". " + server, + server.getLoad() <= max); + assertTrue("All servers should have load no less than " + min + ". " + server, + server.getLoad() >= min); } } @@ -561,7 +563,7 @@ protected void testWithCluster(Map> serverMap, Map>> LoadOfAllTable = (Map) mockClusterServersWithTables(serverMap); List plans = loadBalancer.balanceCluster(LoadOfAllTable); - assertNotNull(plans); + assertNotNull("Initial cluster balance should produce plans.", plans); // Check to see that this actually got to a stable place. if (assertFullyBalanced || assertFullyBalancedForReplicas) { @@ -575,7 +577,8 @@ protected void testWithCluster(Map> serverMap, assertClusterAsBalanced(balancedCluster); LoadOfAllTable = (Map) mockClusterServersWithTables(serverMap); List secondPlans = loadBalancer.balanceCluster(LoadOfAllTable); - assertNull(secondPlans); + assertNull("Given a requirement to be fully balanced, second attempt at plans should " + + "produce none.", secondPlans); } if (assertFullyBalancedForReplicas) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java index 02ace93d6679..0f8852f96f7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.concurrent.atomic.AtomicLong; @@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HStore; @@ -164,7 +166,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // Add the only reference we get for compact MOB case // because new store file will have only one MOB reference // in this case - of newly compacted MOB file - mobRefSet.get().add(mobFileWriter.getPath().getName()); + mobRefSet.get().put(store.getTableName(), mobFileWriter.getPath().getName()); } do { hasMore = scanner.next(cells, scannerContext); @@ -237,9 +239,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (size > mobSizeThreshold) { // If the value size is larger than the threshold, it's regarded as a mob. Since // its value is already in the mob file, directly write this cell to the store file - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); + Optional refTable = MobUtils.getTableName(c); + if (refTable.isPresent()) { + mobRefSet.get().put(refTable.get(), MobUtils.getMobFileName(c)); + writer.append(c); + } else { + throw new IOException(String.format("MOB cell did not contain a tablename " + + "tag. should not be possible. see ref guide on mob troubleshooting. " + + "store={} cell={}", getStoreInfo(), c)); + } } else { // If the value is not larger than the threshold, it's not regarded a mob. Retrieve // the mob cell from the mob file, and write it back to the store file. @@ -255,9 +263,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // directly write the cell to the store file, and leave it to be handled by the // next compaction. LOG.error("Empty value for: " + c); - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); + Optional refTable = MobUtils.getTableName(c); + if (refTable.isPresent()) { + mobRefSet.get().put(refTable.get(), MobUtils.getMobFileName(c)); + writer.append(c); + } else { + throw new IOException(String.format("MOB cell did not contain a tablename " + + "tag. should not be possible. see ref guide on mob troubleshooting. " + + "store={} cell={}", getStoreInfo(), c)); + } } } } else { @@ -280,7 +294,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); // Add ref we get for compact MOB case - mobRefSet.get().add(mobFileWriter.getPath().getName()); + mobRefSet.get().put(store.getTableName(), mobFileWriter.getPath().getName()); } int len = c.getSerializedSize(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java deleted file mode 100644 index b530df3bfb8d..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Random; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.RegionSplitter; -import org.junit.After; -import org.junit.Before; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Mob file compaction base test. - * 1. Enables batch mode for regular MOB compaction, - * Sets batch size to 7 regions. (Optional) - * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec - * 3. Creates MOB table with 20 regions - * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. - * 5. Repeats 4. two more times - * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) - * 7. Runs major MOB compaction. - * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 - * 9. Waits for a period of time larger than minimum age to archive - * 10. Runs Mob cleaner chore - * 11 Verifies that number of MOB files in a mob directory is 20. - * 12 Runs scanner and checks all 3 * 1000 rows. - */ -@SuppressWarnings("deprecation") -public abstract class TestMobCompactionBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestMobCompactionBase.class); - - protected HBaseTestingUtility HTU; - - protected final static String famStr = "f1"; - protected final static byte[] fam = Bytes.toBytes(famStr); - protected final static byte[] qualifier = Bytes.toBytes("q1"); - protected final static long mobLen = 10; - protected final static byte[] mobVal = Bytes - .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); - - protected Configuration conf; - protected TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; - private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; - protected Admin admin; - protected Table table = null; - protected long minAgeToArchive = 10000; - protected int numRegions = 20; - protected int rows = 1000; - - protected MobFileCleanerChore cleanerChore; - - public TestMobCompactionBase() { - } - - - @Before - public void setUp() throws Exception { - HTU = new HBaseTestingUtility(); - tableDescriptor = HTU.createModifyableTableDescriptor(getClass().getName()); - conf = HTU.getConfiguration(); - - initConf(); - - HTU.startMiniCluster(); - admin = HTU.getAdmin(); - cleanerChore = new MobFileCleanerChore(); - familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); - familyDescriptor.setMobEnabled(true); - familyDescriptor.setMobThreshold(mobLen); - familyDescriptor.setMaxVersions(1); - tableDescriptor.setColumnFamily(familyDescriptor); - RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); - byte[][] splitKeys = splitAlgo.split(numRegions); - table = HTU.createTable(tableDescriptor, splitKeys); - - } - - protected void initConf() { - - conf.setInt("hfile.format.version", 3); - // Disable automatic MOB compaction - conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); - // Disable automatic MOB file cleaner chore - conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); - // Set minimum age to archive to 10 sec - conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); - // Set compacted file discharger interval to a half minAgeToArchive - conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); - } - - private void loadData(int num) { - - Random r = new Random(); - try { - LOG.info("Started loading {} rows", num); - for (int i = 0; i < num; i++) { - byte[] key = new byte[32]; - r.nextBytes(key); - Put p = new Put(key); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - } - admin.flush(table.getName()); - LOG.info("Finished loading {} rows", num); - } catch (Exception e) { - LOG.error("MOB file compaction chore test FAILED", e); - fail("MOB file compaction chore test FAILED"); - } - } - - @After - public void tearDown() throws Exception { - admin.disableTable(tableDescriptor.getTableName()); - admin.deleteTable(tableDescriptor.getTableName()); - HTU.shutdownMiniCluster(); - } - - - public void baseTestMobFileCompaction() throws InterruptedException, IOException { - - // Load and flush data 3 times - loadData(rows); - loadData(rows); - loadData(rows); - long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions * 3, num); - // Major MOB compact - mobCompact(admin, tableDescriptor, familyDescriptor); - // wait until compaction is complete - while (admin.getCompactionState(tableDescriptor.getTableName()) != CompactionState.NONE) { - Thread.sleep(100); - } - - num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions * 4, num); - // We have guarantee, that compacted file discharger will run during this pause - // because it has interval less than this wait time - LOG.info("Waiting for {}ms", minAgeToArchive + 1000); - - Thread.sleep(minAgeToArchive + 1000); - LOG.info("Cleaning up MOB files"); - // Cleanup again - cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); - - num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions, num); - - long scanned = scanTable(); - assertEquals(3 * rows, scanned); - - } - - protected abstract void mobCompact(Admin admin2, TableDescriptor tableDescriptor, - ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException; - - - protected long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) - throws IOException { - FileSystem fs = FileSystem.get(conf); - Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); - FileStatus[] stat = fs.listStatus(dir); - for (FileStatus st : stat) { - LOG.debug("MOB Directory content: {}", st.getPath()); - } - LOG.debug("MOB Directory content total files: {}", stat.length); - - return stat.length; - } - - - protected long scanTable() { - try { - - Result result; - ResultScanner scanner = table.getScanner(fam); - long counter = 0; - while ((result = scanner.next()) != null) { - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - counter++; - } - return counter; - } catch (Exception e) { - LOG.error("MOB file compaction test FAILED", e); - if (HTU != null) { - fail(e.getMessage()); - } else { - System.exit(-1); - } - } - return 0; - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java index 414ca3e9cde6..09f7f589b145 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java @@ -20,12 +20,9 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,37 +44,25 @@ */ @SuppressWarnings("deprecation") @Category(LargeTests.class) -public class TestMobCompactionOptMode extends TestMobCompactionBase{ +public class TestMobCompactionOptMode extends TestMobCompactionWithDefaults { private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionOptMode.class); @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMobCompactionOptMode.class); - public TestMobCompactionOptMode() { - } - - @Override - protected void initConf() { - super.initConf(); + @BeforeClass + public static void configureOptimizedCompaction() throws InterruptedException, IOException { + HTU.shutdownMiniHBaseCluster(); conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); - } - - @Test - public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - LOG.info("MOB compaction generational (non-batch) mode started"); - baseTestMobFileCompaction(); - LOG.info("MOB compaction generational (non-batch) mode finished OK"); - + HTU.startMiniHBaseCluster(); } @Override - protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, - ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { - // Major compact MOB table - admin.majorCompact(tableDescriptor.getTableName(), familyDescriptor.getName()); + protected String description() { + return "generational (non-batch) mode"; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java index 45fecc15b1e7..117b9eed28f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java @@ -20,13 +20,12 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,46 +48,43 @@ */ @SuppressWarnings("deprecation") @Category(LargeTests.class) -public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{ +public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionWithDefaults { private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionOptRegionBatchMode.class); @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class); - private int batchSize = 7; + private static final int batchSize = 7; private MobFileCompactionChore compactionChore; - public TestMobCompactionOptRegionBatchMode() { - } - @Before public void setUp() throws Exception { super.setUp(); compactionChore = new MobFileCompactionChore(conf, batchSize); } - protected void initConf() { - super.initConf(); + @BeforeClass + public static void configureOptimizedCompactionAndBatches() + throws InterruptedException, IOException { + HTU.shutdownMiniHBaseCluster(); conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); + HTU.startMiniHBaseCluster(); } @Override - protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + protected void mobCompactImpl(TableDescriptor tableDescriptor, ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { - // Major compact with batch mode enabled + LOG.debug("compacting {} in batch mode.", tableDescriptor.getTableName()); compactionChore.performMajorCompactionInBatches(admin, tableDescriptor, familyDescriptor); } - @Test - public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - LOG.info("MOB compaction chore generational batch mode started"); - baseTestMobFileCompaction(); - LOG.info("MOB compaction chore generational batch mode finished OK"); - + @Override + protected String description() { + return "generational batch mode"; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java deleted file mode 100644 index 01b6804e950a..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob; -import java.io.IOException; - -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Mob file compaction chore in a regular non-batch mode test. - * 1. Uses default (non-batch) mode for regular MOB compaction, - * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec - * 3. Creates MOB table with 20 regions - * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. - * 5. Repeats 4. two more times - * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) - * 7. Runs major MOB compaction. - * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 - * 9. Waits for a period of time larger than minimum age to archive - * 10. Runs Mob cleaner chore - * 11 Verifies that number of MOB files in a mob directory is 20. - * 12 Runs scanner and checks all 3 * 1000 rows. - */ -@SuppressWarnings("deprecation") -@Category(LargeTests.class) -public class TestMobCompactionRegularMode extends TestMobCompactionBase{ - private static final Logger LOG = - LoggerFactory.getLogger(TestMobCompactionRegularMode.class); - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class); - - public TestMobCompactionRegularMode() { - } - - @Override - protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, - ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { - // Major compact MOB table - admin.majorCompact(tableDescriptor.getTableName(), familyDescriptor.getName()); - } - - @Test - public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - LOG.info("MOB compaction regular mode started"); - baseTestMobFileCompaction(); - LOG.info("MOB compaction regular mode finished OK"); - - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java index 819f09552193..51517894f477 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java @@ -20,13 +20,12 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,43 +48,39 @@ */ @SuppressWarnings("deprecation") @Category(LargeTests.class) -public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBase{ +public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionWithDefaults { private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionRegularRegionBatchMode.class); @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class); - private int batchSize = 7; + private static final int batchSize = 7; private MobFileCompactionChore compactionChore; - public TestMobCompactionRegularRegionBatchMode() { - } - @Before public void setUp() throws Exception { super.setUp(); compactionChore = new MobFileCompactionChore(conf, batchSize); } - protected void initConf() { - super.initConf(); + @BeforeClass + public static void configureCompactionBatches() throws InterruptedException, IOException { + HTU.shutdownMiniHBaseCluster(); conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); + HTU.startMiniHBaseCluster(); } @Override - protected void mobCompact(Admin admin, TableDescriptor tableDescriptor, + protected void mobCompactImpl(TableDescriptor tableDescriptor, ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { - // Major compact with batch mode enabled + LOG.debug("compacting {} in batch mode.", tableDescriptor.getTableName()); compactionChore.performMajorCompactionInBatches(admin, tableDescriptor, familyDescriptor); } - @Test - public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - LOG.info("MOB compaction chore regular batch mode started"); - baseTestMobFileCompaction(); - LOG.info("MOB compaction chore regular batch mode finished OK"); - + @Override + protected String description() { + return "regular batch mode"; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java new file mode 100644 index 000000000000..22fb31f3d2b2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java @@ -0,0 +1,335 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RegionSplitter; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction base test. + * 1. Enables batch mode for regular MOB compaction, + * Sets batch size to 7 regions. (Optional) + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 3 * 1000 rows. + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompactionWithDefaults { + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionWithDefaults.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactionWithDefaults.class); + + protected static HBaseTestingUtility HTU; + protected static Configuration conf; + protected static long minAgeToArchive = 10000; + + protected final static String famStr = "f1"; + protected final static byte[] fam = Bytes.toBytes(famStr); + protected final static byte[] qualifier = Bytes.toBytes("q1"); + protected final static long mobLen = 10; + protected final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + @Rule + public TestName test = new TestName(); + protected TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor; + private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor; + protected Admin admin; + protected TableName table = null; + protected int numRegions = 20; + protected int rows = 1000; + + protected MobFileCleanerChore cleanerChore; + + @BeforeClass + public static void htuStart() throws Exception { + HTU = new HBaseTestingUtility(); + conf = HTU.getConfiguration(); + conf.setInt("hfile.format.version", 3); + // Disable automatic MOB compaction + conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); + // Disable automatic MOB file cleaner chore + conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); + // Set minimum age to archive to 10 sec + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); + // Set compacted file discharger interval to a half minAgeToArchive + conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); + conf.setBoolean("hbase.regionserver.compaction.enabled", false); + HTU.startMiniCluster(); + } + + @AfterClass + public static void htuStop() throws Exception { + HTU.shutdownMiniCluster(); + } + + @Before + public void setUp() throws Exception { + tableDescriptor = HTU.createModifyableTableDescriptor(test.getMethodName()); + admin = HTU.getAdmin(); + cleanerChore = new MobFileCleanerChore(); + familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam); + familyDescriptor.setMobEnabled(true); + familyDescriptor.setMobThreshold(mobLen); + familyDescriptor.setMaxVersions(1); + tableDescriptor.setColumnFamily(familyDescriptor); + RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); + byte[][] splitKeys = splitAlgo.split(numRegions); + table = HTU.createTable(tableDescriptor, splitKeys).getName(); + } + + private void loadData(TableName tableName, int num) { + + Random r = new Random(); + LOG.info("Started loading {} rows into {}", num, tableName); + try (final Table table = HTU.getConnection().getTable(tableName)) { + for (int i = 0; i < num; i++) { + byte[] key = new byte[32]; + r.nextBytes(key); + Put p = new Put(key); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + } + admin.flush(tableName); + LOG.info("Finished loading {} rows into {}", num, tableName); + } catch (Exception e) { + LOG.error("MOB file compaction chore test FAILED", e); + fail("MOB file compaction chore test FAILED"); + } + } + + @After + public void tearDown() throws Exception { + admin.disableTable(tableDescriptor.getTableName()); + admin.deleteTable(tableDescriptor.getTableName()); + } + + @Test + public void baseTestMobFileCompaction() throws InterruptedException, IOException { + LOG.info("MOB compaction " + description() + " started"); + loadAndFlushThreeTimes(rows, table, famStr); + mobCompact(tableDescriptor, familyDescriptor); + assertEquals("Should have 4 MOB files per region due to 3xflush + compaction.", numRegions * 4, + getNumberOfMobFiles(table, famStr)); + cleanupAndVerifyCounts(table, famStr, 3*rows); + LOG.info("MOB compaction " + description() + " finished OK"); + } + + @Test + public void testMobFileCompactionAfterSnapshotClone() throws InterruptedException, IOException { + final TableName clone = TableName.valueOf(test.getMethodName() + "-clone"); + LOG.info("MOB compaction of cloned snapshot, " + description() + " started"); + loadAndFlushThreeTimes(rows, table, famStr); + LOG.debug("Taking snapshot and cloning table {}", table); + admin.snapshot(test.getMethodName(), table); + admin.cloneSnapshot(test.getMethodName(), clone); + assertEquals("Should have 3 hlinks per region in MOB area from snapshot clone", 3 * numRegions, + getNumberOfMobFiles(clone, famStr)); + mobCompact(admin.getDescriptor(clone), familyDescriptor); + assertEquals("Should have 3 hlinks + 1 MOB file per region due to clone + compact", + 4 * numRegions, getNumberOfMobFiles(clone, famStr)); + cleanupAndVerifyCounts(clone, famStr, 3*rows); + LOG.info("MOB compaction of cloned snapshot, " + description() + " finished OK"); + } + + @Test + public void testMobFileCompactionAfterSnapshotCloneAndFlush() throws InterruptedException, + IOException { + final TableName clone = TableName.valueOf(test.getMethodName() + "-clone"); + LOG.info("MOB compaction of cloned snapshot after flush, " + description() + " started"); + loadAndFlushThreeTimes(rows, table, famStr); + LOG.debug("Taking snapshot and cloning table {}", table); + admin.snapshot(test.getMethodName(), table); + admin.cloneSnapshot(test.getMethodName(), clone); + assertEquals("Should have 3 hlinks per region in MOB area from snapshot clone", 3 * numRegions, + getNumberOfMobFiles(clone, famStr)); + loadAndFlushThreeTimes(rows, clone, famStr); + mobCompact(admin.getDescriptor(clone), familyDescriptor); + assertEquals("Should have 7 MOB file per region due to clone + 3xflush + compact", + 7 * numRegions, getNumberOfMobFiles(clone, famStr)); + cleanupAndVerifyCounts(clone, famStr, 6*rows); + LOG.info("MOB compaction of cloned snapshot w flush, " + description() + " finished OK"); + } + + protected void loadAndFlushThreeTimes(int rows, TableName table, String family) + throws IOException { + final long start = getNumberOfMobFiles(table, family); + // Load and flush data 3 times + loadData(table, rows); + loadData(table, rows); + loadData(table, rows); + assertEquals("Should have 3 more mob files per region from flushing.", start + numRegions * 3, + getNumberOfMobFiles(table, family)); + } + + protected String description() { + return "regular mode"; + } + + protected void enableCompactions() throws IOException { + final List serverList = admin.getRegionServers().stream().map(sn -> sn.getServerName()) + .collect(Collectors.toList()); + admin.compactionSwitch(true, serverList); + } + + protected void disableCompactions() throws IOException { + final List serverList = admin.getRegionServers().stream().map(sn -> sn.getServerName()) + .collect(Collectors.toList()); + admin.compactionSwitch(false, serverList); + } + + /** + * compact the given table and return once it is done. + * should presume compactions are disabled when called. + * should ensure compactions are disabled before returning. + */ + protected void mobCompact(TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { + LOG.debug("Major compact MOB table " + tableDescriptor.getTableName()); + enableCompactions(); + mobCompactImpl(tableDescriptor, familyDescriptor); + waitUntilCompactionIsComplete(tableDescriptor.getTableName()); + disableCompactions(); + } + + /** + * Call the API for compaction specific to the test set. + * should not wait for compactions to finish. + * may assume compactions are enabled when called. + */ + protected void mobCompactImpl(TableDescriptor tableDescriptor, + ColumnFamilyDescriptor familyDescriptor) throws IOException, InterruptedException { + admin.majorCompact(tableDescriptor.getTableName(), familyDescriptor.getName()); + } + + protected void waitUntilCompactionIsComplete(TableName table) + throws IOException, InterruptedException { + CompactionState state = admin.getCompactionState(table); + while (state != CompactionState.NONE) { + LOG.debug("Waiting for compaction on {} to complete. current state {}", table, state); + Thread.sleep(100); + state = admin.getCompactionState(table); + } + LOG.debug("done waiting for compaction on {}", table); + } + + protected void cleanupAndVerifyCounts(TableName table, String family, int rows) + throws InterruptedException, IOException { + // We have guarantee, that compacted file discharger will run during this pause + // because it has interval less than this wait time + LOG.info("Waiting for {}ms", minAgeToArchive + 1000); + + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + cleanerChore.cleanupObsoleteMobFiles(conf, table); + + assertEquals("After cleaning, we should have 1 MOB file per region based on size.", numRegions, + getNumberOfMobFiles(table, family)); + + LOG.debug("checking count of rows"); + long scanned = scanTable(table); + assertEquals("Got the wrong number of rows in table " + table + " cf " + family, rows, scanned); + + } + + protected long getNumberOfMobFiles(TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + + + protected long scanTable(TableName tableName) { + try (final Table table = HTU.getConnection().getTable(tableName); + final ResultScanner scanner = table.getScanner(fam)) { + Result result; + long counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + counter++; + } + return counter; + } catch (Exception e) { + LOG.error("MOB file compaction test FAILED", e); + if (HTU != null) { + fail(e.getMessage()); + } else { + System.exit(-1); + } + } + return 0; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index a96f1561777e..4e153ae7c58c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -139,37 +139,36 @@ public void testWholesomeMerge() throws Exception { LOG.info("Starting " + name.getMethodName()); final TableName tableName = TableName.valueOf(name.getMethodName()); - // Create table and load data. - Table table = createTableAndLoadData(MASTER, tableName); - // Merge 1st and 2nd region - mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1, - INITIAL_REGION_NUM - 1); + try { + // Create table and load data. + Table table = createTableAndLoadData(MASTER, tableName); + // Merge 1st and 2nd region + mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1, INITIAL_REGION_NUM - 1); - // Merge 2nd and 3th region - PairOfSameType mergedRegions = - mergeRegionsAndVerifyRegionNum(MASTER, tableName, 1, 2, - INITIAL_REGION_NUM - 2); + // Merge 2nd and 3th region + PairOfSameType mergedRegions = + mergeRegionsAndVerifyRegionNum(MASTER, tableName, 1, 2, INITIAL_REGION_NUM - 2); - verifyRowCount(table, ROWSIZE); + verifyRowCount(table, ROWSIZE); - // Randomly choose one of the two merged regions - RegionInfo hri = RandomUtils.nextBoolean() ? - mergedRegions.getFirst() : mergedRegions.getSecond(); - MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); - AssignmentManager am = cluster.getMaster().getAssignmentManager(); - RegionStates regionStates = am.getRegionStates(); + // Randomly choose one of the two merged regions + RegionInfo hri = RandomUtils.nextBoolean() ? mergedRegions.getFirst() : mergedRegions.getSecond(); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + AssignmentManager am = cluster.getMaster().getAssignmentManager(); + RegionStates regionStates = am.getRegionStates(); - // We should not be able to assign it again - am.assign(hri); - assertFalse("Merged region can't be assigned", - regionStates.isRegionInTransition(hri)); + // We should not be able to assign it again + am.assign(hri); + assertFalse("Merged region can't be assigned", regionStates.isRegionInTransition(hri)); - // We should not be able to unassign it either - am.unassign(hri); - assertFalse("Merged region can't be unassigned", - regionStates.isRegionInTransition(hri)); + // We should not be able to unassign it either + am.unassign(hri); + assertFalse("Merged region can't be unassigned", regionStates.isRegionInTransition(hri)); - table.close(); + table.close(); + } finally { + TEST_UTIL.deleteTable(tableName); + } } /** @@ -180,27 +179,31 @@ public void testWholesomeMerge() throws Exception { public void testMergeAndRestartingMaster() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - // Create table and load data. - Table table = createTableAndLoadData(MASTER, tableName); - try { - MyMasterRpcServices.enabled.set(true); + // Create table and load data. + Table table = createTableAndLoadData(MASTER, tableName); - // Merge 1st and 2nd region - mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1, INITIAL_REGION_NUM - 1); + try { + MyMasterRpcServices.enabled.set(true); + + // Merge 1st and 2nd region + mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1, INITIAL_REGION_NUM - 1); + } finally { + MyMasterRpcServices.enabled.set(false); + } + + table.close(); } finally { - MyMasterRpcServices.enabled.set(false); + TEST_UTIL.deleteTable(tableName); } - - table.close(); } @Test public void testCleanMergeReference() throws Exception { LOG.info("Starting " + name.getMethodName()); ADMIN.catalogJanitorSwitch(false); + final TableName tableName = TableName.valueOf(name.getMethodName()); try { - final TableName tableName = TableName.valueOf(name.getMethodName()); // Create table and load data. Table table = createTableAndLoadData(MASTER, tableName); // Merge 1st and 2nd region @@ -303,6 +306,7 @@ public void testCleanMergeReference() throws Exception { assertFalse(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells())); } finally { ADMIN.catalogJanitorSwitch(true); + TEST_UTIL.deleteTable(tableName); } } @@ -365,40 +369,39 @@ public void testMerge() throws Exception { @Test public void testMergeWithReplicas() throws Exception { final TableName tableName = TableName.valueOf(name.getMethodName()); - // Create table and load data. - createTableAndLoadData(MASTER, tableName, 5, 2); - List> initialRegionToServers = - MetaTableAccessor.getTableRegionsAndLocations( - TEST_UTIL.getConnection(), tableName); - // Merge 1st and 2nd region - PairOfSameType mergedRegions = mergeRegionsAndVerifyRegionNum(MASTER, tableName, - 0, 2, 5 * 2 - 2); - List> currentRegionToServers = - MetaTableAccessor.getTableRegionsAndLocations( - TEST_UTIL.getConnection(), tableName); - List initialRegions = new ArrayList<>(); - for (Pair p : initialRegionToServers) { - initialRegions.add(p.getFirst()); - } - List currentRegions = new ArrayList<>(); - for (Pair p : currentRegionToServers) { - currentRegions.add(p.getFirst()); + try { + // Create table and load data. + Table table = createTableAndLoadData(MASTER, tableName, 5, 2); + List> initialRegionToServers = + MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName); + // Merge 1st and 2nd region + PairOfSameType mergedRegions = + mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 2, 5 * 2 - 2); + List> currentRegionToServers = + MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName); + List initialRegions = new ArrayList<>(); + for (Pair p : initialRegionToServers) { + initialRegions.add(p.getFirst()); + } + List currentRegions = new ArrayList<>(); + for (Pair p : currentRegionToServers) { + currentRegions.add(p.getFirst()); + } + assertTrue(initialRegions.contains(mergedRegions.getFirst())); //this is the first region + assertTrue(initialRegions.contains(RegionReplicaUtil + .getRegionInfoForReplica(mergedRegions.getFirst(), 1))); //this is the replica of the first region + assertTrue(initialRegions.contains(mergedRegions.getSecond())); //this is the second region + assertTrue(initialRegions.contains(RegionReplicaUtil + .getRegionInfoForReplica(mergedRegions.getSecond(), 1))); //this is the replica of the second region + assertTrue(!initialRegions.contains(currentRegions.get(0))); //this is the new region + assertTrue(!initialRegions.contains(RegionReplicaUtil.getRegionInfoForReplica(currentRegions.get(0), 1))); //replica of the new region + assertTrue(currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica(currentRegions.get(0), 1))); //replica of the new region + assertTrue(!currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica(mergedRegions.getFirst(), 1))); //replica of the merged region + assertTrue(!currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica(mergedRegions.getSecond(), 1))); //replica of the merged region + table.close(); + } finally { + TEST_UTIL.deleteTable(tableName); } - assertTrue(initialRegions.contains(mergedRegions.getFirst())); //this is the first region - assertTrue(initialRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - mergedRegions.getFirst(), 1))); //this is the replica of the first region - assertTrue(initialRegions.contains(mergedRegions.getSecond())); //this is the second region - assertTrue(initialRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - mergedRegions.getSecond(), 1))); //this is the replica of the second region - assertTrue(!initialRegions.contains(currentRegions.get(0))); //this is the new region - assertTrue(!initialRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - currentRegions.get(0), 1))); //replica of the new region - assertTrue(currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - currentRegions.get(0), 1))); //replica of the new region - assertTrue(!currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - mergedRegions.getFirst(), 1))); //replica of the merged region - assertTrue(!currentRegions.contains(RegionReplicaUtil.getRegionInfoForReplica( - mergedRegions.getSecond(), 1))); //replica of the merged region } private PairOfSameType mergeRegionsAndVerifyRegionNum( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java index 0b4229aa0862..562e0ca190a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestSnapshotScannerHDFSAclController.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -139,6 +140,13 @@ public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } + private void snapshotAndWait(final String snapShotName, final TableName tableName) + throws Exception{ + admin.snapshot(snapShotName, tableName); + LOG.info("Sleep for three seconds, waiting for HDFS Acl setup"); + Threads.sleep(3000); + } + @Test public void testGrantGlobal1() throws Exception { final String grantUserName = name.getMethodName(); @@ -149,7 +157,7 @@ public void testGrantGlobal1() throws Exception { String snapshot2 = namespace + "s2"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot1, table); + snapshotAndWait(snapshot1, table); // grant G(R) SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); @@ -167,7 +175,7 @@ public void testGrantGlobal1() throws Exception { SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); // take a snapshot and ACLs are inherited automatically - admin.snapshot(snapshot2, table); + snapshotAndWait(snapshot2, table); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 6); assertTrue(hasUserGlobalHdfsAcl(aclTable, grantUserName)); deleteTable(table); @@ -188,7 +196,7 @@ public void testGrantGlobal2() throws Exception { SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); // create table in namespace1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); admin.grant(new UserPermission(grantUserName, Permission.newBuilder(namespace1).withActions(READ).build()), false); @@ -196,7 +204,7 @@ public void testGrantGlobal2() throws Exception { SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, WRITE); // create table in namespace2 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table2); - admin.snapshot(snapshot2, table2); + snapshotAndWait(snapshot2, table2); // check scan snapshot TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, -1); @@ -223,12 +231,12 @@ public void testGrantGlobal3() throws Exception { SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); // grant table1(R) TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table1, READ); // grant G(W) SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, WRITE); TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table2); - admin.snapshot(snapshot2, table2); + snapshotAndWait(snapshot2, table2); // check scan snapshot TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, -1); @@ -255,12 +263,12 @@ public void testGrantNamespace1() throws Exception { // create table1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // grant N(R) SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); // create table2 and snapshot, ACLs can be inherited automatically TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table2); - admin.snapshot(snapshot2, table2); + snapshotAndWait(snapshot2, table2); // check scan snapshot TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 6); @@ -287,7 +295,7 @@ public void testGrantNamespace2() throws Exception { // create table1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // grant N(R) SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); @@ -314,7 +322,7 @@ public void testGrantNamespace3() throws Exception { // create table1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant namespace(R) SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); // grant global(R) @@ -343,7 +351,7 @@ public void testGrantTable() throws Exception { LOG.info("Create table"); try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, table1)) { TestHDFSAclHelper.put(t); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // table owner can scan table snapshot LOG.info("Scan snapshot"); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, @@ -356,7 +364,8 @@ public void testGrantTable() throws Exception { // grant table1(R) TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table1, READ); TestHDFSAclHelper.put2(t); - admin.snapshot(snapshot2, table1); + snapshotAndWait(snapshot2, table1); + // check scan snapshot TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, 6); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot2, 10); @@ -389,7 +398,7 @@ public void testGrantMobTable() throws Exception { try (Table t = TestHDFSAclHelper.createMobTable(TEST_UTIL, table)) { TestHDFSAclHelper.put(t); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6); assertTrue(hasUserTableHdfsAcl(aclTable, grantUserName, table)); @@ -407,7 +416,7 @@ public void testRevokeGlobal1() throws Exception { String snapshot1 = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot1, -1); @@ -425,7 +434,7 @@ public void testRevokeGlobal2() throws Exception { String snapshot1 = namespace + "s1"; TableName table1 = TableName.valueOf(namespace, name.getMethodName()); TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // grant G(R), grant N(R), grant T(R) -> revoke G(R) SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); @@ -450,7 +459,7 @@ public void testRevokeGlobal3() throws Exception { TableName table1 = TableName.valueOf(namespace, name.getMethodName()); String snapshot1 = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // grant G(R), grant T(R) -> revoke G(R) SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ); @@ -475,7 +484,7 @@ public void testRevokeNamespace1() throws Exception { TableName table1 = TableName.valueOf(namespace, name.getMethodName()); String snapshot1 = namespace + "s1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table1); - admin.snapshot(snapshot1, table1); + snapshotAndWait(snapshot1, table1); // revoke N(R) SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); @@ -504,7 +513,7 @@ public void testRevokeNamespace2() throws Exception { TableName table = TableName.valueOf(namespace, name.getMethodName()); String snapshot = namespace + "s1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant N(R), grant T(R) -> revoke N(R) SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); @@ -528,7 +537,7 @@ public void testRevokeTable1() throws Exception { TableName table = TableName.valueOf(namespace, name.getMethodName()); String snapshot = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant T(R) -> revoke table family TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ); @@ -553,7 +562,7 @@ public void testRevokeTable2() throws Exception { TableName table = TableName.valueOf(namespace, name.getMethodName()); String snapshot = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant T(R), grant N(R) -> revoke T(R) TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ); @@ -575,7 +584,7 @@ public void testRevokeTable3() throws Exception { TableName table = TableName.valueOf(namespace, name.getMethodName()); String snapshot = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant T(R), grant G(R) -> revoke T(R) TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ); @@ -603,7 +612,7 @@ public void testTruncateTable() throws Exception { try (Table t = TestHDFSAclHelper.createTable(TEST_UTIL, tableName)) { TestHDFSAclHelper.put(t); // snapshot - admin.snapshot(snapshot, tableName); + snapshotAndWait(snapshot, tableName); // grant user2 namespace permission SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName2, namespace, READ); // grant user table permission @@ -613,7 +622,7 @@ public void testTruncateTable() throws Exception { admin.truncateTable(tableName, true); TestHDFSAclHelper.put2(t); // snapshot - admin.snapshot(snapshot2, tableName); + snapshotAndWait(snapshot2, tableName); // check scan snapshot TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser2, snapshot, 6); @@ -640,7 +649,7 @@ public void testDeleteTable() throws Exception { TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); // snapshot - admin.snapshot(snapshot1, table); + snapshotAndWait(snapshot1, table); // grant user table permission TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName1, table, READ); SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName2, namespace, READ); @@ -675,7 +684,7 @@ public void testDeleteNamespace() throws Exception { String snapshot = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); // snapshot - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); // grant namespace permission SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ); // delete table @@ -704,7 +713,7 @@ public void testCleanArchiveTableDir() throws Exception { String snapshot = namespace + "t1"; TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); TestHDFSAclHelper.grantOnTable(TEST_UTIL, grantUserName, table, READ); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6); @@ -748,7 +757,7 @@ public void testModifyTable1() throws Exception { TestHDFSAclHelper.createNamespace(TEST_UTIL, namespace); SecureTestUtil.grantOnNamespace(TEST_UTIL, nsUserName, namespace, READ); TableDescriptor td = TestHDFSAclHelper.createUserScanSnapshotDisabledTable(TEST_UTIL, table); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); SecureTestUtil.grantGlobal(TEST_UTIL, globalUserName2, READ); TestHDFSAclHelper.grantOnTable(TEST_UTIL, tableUserName, table, READ); SecureTestUtil.grantOnTable(TEST_UTIL, tableUserName2, table, TestHDFSAclHelper.COLUMN1, null, @@ -896,7 +905,7 @@ public void testRestartMaster() throws Exception { // create table1 and snapshot TestHDFSAclHelper.createTableAndPut(TEST_UTIL, table); aclTable = TEST_UTIL.getConnection().getTable(PermissionStorage.ACL_TABLE_NAME); - admin.snapshot(snapshot, table); + snapshotAndWait(snapshot, table); TestHDFSAclHelper.canUserScanSnapshot(TEST_UTIL, grantUser, snapshot, 6); deleteTable(table); deleteTable(table2); diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml index 21066b2039de..027fb11cc6c3 100644 --- a/src/site/xdoc/downloads.xml +++ b/src/site/xdoc/downloads.xml @@ -68,24 +68,24 @@ under the License. - 2.1.9 + 2.1.10 - 2020/02/17 + 2020/04/08 - 2.1.8 vs 2.1.9 + 2.1.9 vs 2.1.10 - Changes + Changes - Release Notes + Release Notes - src (sha512 asc)
- bin (sha512 asc)
- client-bin (sha512 asc) + src (sha512 asc)
+ bin (sha512 asc)
+ client-bin (sha512 asc)