test-patch.sh 68 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466
  1. #!/usr/bin/env bash
  2. # Licensed to the Apache Software Foundation (ASF) under one or more
  3. # contributor license agreements. See the NOTICE file distributed with
  4. # this work for additional information regarding copyright ownership.
  5. # The ASF licenses this file to You under the Apache License, Version 2.0
  6. # (the "License"); you may not use this file except in compliance with
  7. # the License. You may obtain a copy of the License at
  8. #
  9. # http://www.apache.org/licenses/LICENSE-2.0
  10. #
  11. # Unless required by applicable law or agreed to in writing, software
  12. # distributed under the License is distributed on an "AS IS" BASIS,
  13. # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  14. # See the License for the specific language governing permissions and
  15. # limitations under the License.
  16. ### BUILD_URL is set by Hudson if it is run by patch process
  17. this="${BASH_SOURCE-$0}"
  18. BINDIR=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P)
  19. CWD=$(pwd)
  20. USER_PARAMS=("$@")
  21. GLOBALTIMER=$(date +"%s")
  22. ## @description Setup the default global variables
  23. ## @audience public
  24. ## @stability stable
  25. ## @replaceable no
  26. function setup_defaults
  27. {
  28. if [[ -z "${MAVEN_HOME:-}" ]]; then
  29. MVN=mvn
  30. else
  31. MVN=${MAVEN_HOME}/bin/mvn
  32. fi
  33. PROJECT_NAME=hadoop
  34. JENKINS=false
  35. PATCH_DIR=/tmp/${PROJECT_NAME}-test-patch/$$
  36. BASEDIR=$(pwd)
  37. FINDBUGS_HOME=${FINDBUGS_HOME:-}
  38. ECLIPSE_HOME=${ECLIPSE_HOME:-}
  39. BUILD_NATIVE=${BUILD_NATIVE:-true}
  40. PATCH_BRANCH=""
  41. CHANGED_MODULES=""
  42. USER_MODULE_LIST=""
  43. OFFLINE=false
  44. CHANGED_FILES=""
  45. REEXECED=false
  46. RESETREPO=false
  47. ISSUE=""
  48. ISSUE_RE='^(HADOOP|YARN|MAPREDUCE|HDFS)-[0-9]+$'
  49. TIMER=$(date +"%s")
  50. PATCHURL=""
  51. OSTYPE=$(uname -s)
  52. # Solaris needs POSIX, not SVID
  53. case ${OSTYPE} in
  54. SunOS)
  55. PS=${PS:-ps}
  56. AWK=${AWK:-/usr/xpg4/bin/awk}
  57. SED=${SED:-/usr/xpg4/bin/sed}
  58. WGET=${WGET:-wget}
  59. GIT=${GIT:-git}
  60. EGREP=${EGREP:-/usr/xpg4/bin/egrep}
  61. GREP=${GREP:-/usr/xpg4/bin/grep}
  62. PATCH=${PATCH:-patch}
  63. DIFF=${DIFF:-/usr/gnu/bin/diff}
  64. JIRACLI=${JIRA:-jira}
  65. ;;
  66. *)
  67. PS=${PS:-ps}
  68. AWK=${AWK:-awk}
  69. SED=${SED:-sed}
  70. WGET=${WGET:-wget}
  71. GIT=${GIT:-git}
  72. EGREP=${EGREP:-egrep}
  73. GREP=${GREP:-grep}
  74. PATCH=${PATCH:-patch}
  75. DIFF=${DIFF:-diff}
  76. JIRACLI=${JIRA:-jira}
  77. ;;
  78. esac
  79. declare -a JIRA_COMMENT_TABLE
  80. declare -a JIRA_FOOTER_TABLE
  81. declare -a JIRA_HEADER
  82. declare -a JIRA_TEST_TABLE
  83. JFC=0
  84. JTC=0
  85. JTT=0
  86. RESULT=0
  87. }
  88. ## @description Print a message to stderr
  89. ## @audience public
  90. ## @stability stable
  91. ## @replaceable no
  92. ## @param string
  93. function hadoop_error
  94. {
  95. echo "$*" 1>&2
  96. }
  97. ## @description Print a message to stderr if --debug is turned on
  98. ## @audience public
  99. ## @stability stable
  100. ## @replaceable no
  101. ## @param string
  102. function hadoop_debug
  103. {
  104. if [[ -n "${HADOOP_SHELL_SCRIPT_DEBUG}" ]]; then
  105. echo "[$(date) DEBUG]: $*" 1>&2
  106. fi
  107. }
  108. ## @description Activate the local timer
  109. ## @audience public
  110. ## @stability stable
  111. ## @replaceable no
  112. function start_clock
  113. {
  114. hadoop_debug "Start clock"
  115. TIMER=$(date +"%s")
  116. }
  117. ## @description Print the elapsed time in seconds since the start of the local timer
  118. ## @audience public
  119. ## @stability stable
  120. ## @replaceable no
  121. function stop_clock
  122. {
  123. local -r stoptime=$(date +"%s")
  124. local -r elapsed=$((stoptime-TIMER))
  125. hadoop_debug "Stop clock"
  126. echo ${elapsed}
  127. }
  128. ## @description Print the elapsed time in seconds since the start of the global timer
  129. ## @audience private
  130. ## @stability stable
  131. ## @replaceable no
  132. function stop_global_clock
  133. {
  134. local -r stoptime=$(date +"%s")
  135. local -r elapsed=$((stoptime-GLOBALTIMER))
  136. hadoop_debug "Stop global clock"
  137. echo ${elapsed}
  138. }
  139. ## @description Add time to the local timer
  140. ## @audience public
  141. ## @stability stable
  142. ## @replaceable no
  143. ## @param seconds
  144. function offset_clock
  145. {
  146. ((TIMER=TIMER-$1))
  147. }
  148. ## @description Add to the header of the display
  149. ## @audience public
  150. ## @stability stable
  151. ## @replaceable no
  152. ## @param string
  153. function add_jira_header
  154. {
  155. JIRA_HEADER[${JHC}]="| $* |"
  156. JHC=$(( JHC+1 ))
  157. }
  158. ## @description Add to the output table. If the first parameter is a number
  159. ## @description that is the vote for that column and calculates the elapsed time
  160. ## @description based upon the last start_clock(). If it the string null, then it is
  161. ## @description a special entry that signifies extra
  162. ## @description content for the final column. The second parameter is the reporting
  163. ## @description subsystem (or test) that is providing the vote. The second parameter
  164. ## @description is always required. The third parameter is any extra verbage that goes
  165. ## @description with that subsystem.
  166. ## @audience public
  167. ## @stability stable
  168. ## @replaceable no
  169. ## @param +1/0/-1/null
  170. ## @param subsystem
  171. ## @param string
  172. ## @return Elapsed time display
  173. function add_jira_table
  174. {
  175. local value=$1
  176. local subsystem=$2
  177. shift 2
  178. local color
  179. local calctime=0
  180. local -r elapsed=$(stop_clock)
  181. if [[ ${elapsed} -lt 0 ]]; then
  182. calctime="N/A"
  183. else
  184. printf -v calctime "%3sm %02ss" $((elapsed/60)) $((elapsed%60))
  185. fi
  186. echo ""
  187. echo "Elapsed time: ${calctime}"
  188. echo ""
  189. case ${value} in
  190. 1|+1)
  191. value="+1"
  192. color="green"
  193. ;;
  194. -1)
  195. color="red"
  196. ;;
  197. 0)
  198. color="blue"
  199. ;;
  200. null)
  201. ;;
  202. esac
  203. if [[ -z ${color} ]]; then
  204. JIRA_COMMENT_TABLE[${JTC}]="| | ${subsystem} | | ${*:-} |"
  205. JTC=$(( JTC+1 ))
  206. else
  207. JIRA_COMMENT_TABLE[${JTC}]="| {color:${color}}${value}{color} | ${subsystem} | ${calctime} | $* |"
  208. JTC=$(( JTC+1 ))
  209. fi
  210. }
  211. ## @description Put the final environment information at the bottom
  212. ## @description of the footer table
  213. ## @stability stable
  214. ## @audience private
  215. ## @replaceable yes
  216. function close_jira_footer
  217. {
  218. # shellcheck disable=SC2016
  219. local -r javaversion=$("${JAVA_HOME}/bin/java" -version 2>&1 | head -1 | ${AWK} '{print $NF}' | tr -d \")
  220. local -r unamea=$(uname -a)
  221. add_jira_footer "Java" "${javaversion}"
  222. add_jira_footer "uname" "${unamea}"
  223. }
  224. ## @description Put the final elapsed time at the bottom of the table.
  225. ## @audience private
  226. ## @stability stable
  227. ## @replaceable no
  228. function close_jira_table
  229. {
  230. local -r elapsed=$(stop_global_clock)
  231. if [[ ${elapsed} -lt 0 ]]; then
  232. calctime="N/A"
  233. else
  234. printf -v calctime "%3sm %02ss" $((elapsed/60)) $((elapsed%60))
  235. fi
  236. echo ""
  237. echo "Total Elapsed time: ${calctime}"
  238. echo ""
  239. JIRA_COMMENT_TABLE[${JTC}]="| | | ${calctime} | |"
  240. JTC=$(( JTC+1 ))
  241. }
  242. ## @description Add to the footer of the display. @@BASE@@ will get replaced with the
  243. ## @description correct location for the local filesystem in dev mode or the URL for
  244. ## @description Jenkins mode.
  245. ## @audience public
  246. ## @stability stable
  247. ## @replaceable no
  248. ## @param subsystem
  249. ## @param string
  250. function add_jira_footer
  251. {
  252. local subsystem=$1
  253. shift 1
  254. JIRA_FOOTER_TABLE[${JFC}]="| ${subsystem} | $* |"
  255. JFC=$(( JFC+1 ))
  256. }
  257. ## @description Special table just for unit test failures
  258. ## @audience public
  259. ## @stability stable
  260. ## @replaceable no
  261. ## @param failurereason
  262. ## @param testlist
  263. function add_jira_test_table
  264. {
  265. local failure=$1
  266. shift 1
  267. JIRA_TEST_TABLE[${JTT}]="| ${failure} | $* |"
  268. JTT=$(( JTT+1 ))
  269. }
  270. ## @description Large display for the user console
  271. ## @audience public
  272. ## @stability stable
  273. ## @replaceable no
  274. ## @param string
  275. ## @return large chunk of text
  276. function big_console_header
  277. {
  278. local text="$*"
  279. local spacing=$(( (75+${#text}) /2 ))
  280. printf "\n\n"
  281. echo "============================================================================"
  282. echo "============================================================================"
  283. printf "%*s\n" ${spacing} "${text}"
  284. echo "============================================================================"
  285. echo "============================================================================"
  286. printf "\n\n"
  287. }
  288. ## @description Remove {color} tags from a string
  289. ## @audience public
  290. ## @stability stable
  291. ## @replaceable no
  292. ## @param string
  293. ## @return string
  294. function colorstripper
  295. {
  296. local string=$1
  297. shift 1
  298. local green=""
  299. local white=""
  300. local red=""
  301. local blue=""
  302. echo "${string}" | \
  303. ${SED} -e "s,{color:red},${red},g" \
  304. -e "s,{color:green},${green},g" \
  305. -e "s,{color:blue},${blue},g" \
  306. -e "s,{color},${white},g"
  307. }
  308. ## @description Find the largest size of a column of an array
  309. ## @audience private
  310. ## @stability evolving
  311. ## @replaceable no
  312. ## @return size
  313. function findlargest
  314. {
  315. local column=$1
  316. shift
  317. local a=("$@")
  318. local sizeofa=${#a[@]}
  319. local i=0
  320. until [[ ${i} -gt ${sizeofa} ]]; do
  321. # shellcheck disable=SC2086
  322. string=$( echo ${a[$i]} | cut -f$((column + 1)) -d\| )
  323. if [[ ${#string} -gt $maxlen ]]; then
  324. maxlen=${#string}
  325. fi
  326. i=$((i+1))
  327. done
  328. echo "${maxlen}"
  329. }
  330. ## @description Verify that ${JAVA_HOME} is defined
  331. ## @audience public
  332. ## @stability stable
  333. ## @replaceable no
  334. ## @return 1 - no JAVA_HOME
  335. ## @return 0 - JAVA_HOME defined
  336. function find_java_home
  337. {
  338. start_clock
  339. if [[ -z ${JAVA_HOME:-} ]]; then
  340. case $(uname -s) in
  341. Darwin)
  342. if [[ -z "${JAVA_HOME}" ]]; then
  343. if [[ -x /usr/libexec/java_home ]]; then
  344. JAVA_HOME="$(/usr/libexec/java_home)"
  345. export JAVA_HOME
  346. else
  347. export JAVA_HOME=/Library/Java/Home
  348. fi
  349. fi
  350. ;;
  351. *)
  352. ;;
  353. esac
  354. fi
  355. if [[ -z ${JAVA_HOME:-} ]]; then
  356. echo "JAVA_HOME is not defined."
  357. add_jira_table -1 pre-patch "JAVA_HOME is not defined."
  358. return 1
  359. fi
  360. return 0
  361. }
  362. ## @description Write the contents of a file to jenkins
  363. ## @params filename
  364. ## @stability stable
  365. ## @audience public
  366. ## @returns ${JIRACLI} exit code
  367. function write_to_jira
  368. {
  369. local -r commentfile=${1}
  370. shift
  371. local retval
  372. if [[ ${OFFLINE} == false
  373. && ${JENKINS} == true ]]; then
  374. export USER=hudson
  375. # shellcheck disable=SC2086
  376. ${JIRACLI} --comment "$(cat ${commentfile})" \
  377. -s https://issues.apache.org/jira \
  378. -a addcomment -u hadoopqa \
  379. -p "${JIRA_PASSWD}" \
  380. --issue "${ISSUE}"
  381. retval=$?
  382. ${JIRACLI} -s https://issues.apache.org/jira \
  383. -a logout -u hadoopqa \
  384. -p "${JIRA_PASSWD}"
  385. fi
  386. return ${retval}
  387. }
  388. ## @description Verify that the patch directory is still in working order
  389. ## @description since bad actors on some systems wipe it out. If not,
  390. ## @description recreate it and then exit
  391. ## @audience private
  392. ## @stability evolving
  393. ## @replaceable yes
  394. ## @returns may exit on failure
  395. function verify_patchdir_still_exists
  396. {
  397. local -r commentfile=/tmp/testpatch.$$.${RANDOM}
  398. local extra=""
  399. if [[ ! -d ${PATCH_DIR} ]]; then
  400. rm "${commentfile}" 2>/dev/null
  401. echo "(!) The patch artifact directory has been removed! " > "${commentfile}"
  402. echo "This is a fatal error for test-patch.sh. Aborting. " >> "${commentfile}"
  403. echo
  404. cat ${commentfile}
  405. echo
  406. if [[ ${JENKINS} == true ]]; then
  407. if [[ -n ${NODE_NAME} ]]; then
  408. extra=" (node ${NODE_NAME})"
  409. fi
  410. echo "Jenkins${extra} information at ${BUILD_URL} may provide some hints. " >> "${commentfile}"
  411. write_to_jira ${commentfile}
  412. fi
  413. rm "${commentfile}"
  414. cleanup_and_exit ${RESULT}
  415. fi
  416. }
  417. ## @description generate a list of all files and line numbers that
  418. ## @description that were added/changed in the source repo
  419. ## @audience private
  420. ## @stability stable
  421. ## @params filename
  422. ## @replaceable no
  423. function compute_gitdiff
  424. {
  425. local outfile=$1
  426. local file
  427. local line
  428. local startline
  429. local counter
  430. local numlines
  431. local actual
  432. pushd "${BASEDIR}" >/dev/null
  433. while read line; do
  434. if [[ ${line} =~ ^\+\+\+ ]]; then
  435. file="./"$(echo "${line}" | cut -f2- -d/)
  436. continue
  437. elif [[ ${line} =~ ^@@ ]]; then
  438. startline=$(echo "${line}" | cut -f3 -d' ' | cut -f1 -d, | tr -d + )
  439. numlines=$(echo "${line}" | cut -f3 -d' ' | cut -s -f2 -d, )
  440. # if this is empty, then just this line
  441. # if it is 0, then no lines were added and this part of the patch
  442. # is strictly a delete
  443. if [[ ${numlines} == 0 ]]; then
  444. continue
  445. elif [[ -z ${numlines} ]]; then
  446. numlines=1
  447. fi
  448. counter=0
  449. until [[ ${counter} -gt ${numlines} ]]; do
  450. ((actual=counter+startline))
  451. echo "${file}:${actual}:" >> "${outfile}"
  452. ((counter=counter+1))
  453. done
  454. fi
  455. done < <("${GIT}" diff --unified=0 --no-color)
  456. popd >/dev/null
  457. }
  458. ## @description Print the command to be executing to the screen. Then
  459. ## @description run the command, sending stdout and stderr to the given filename
  460. ## @description This will also ensure that any directories in ${BASEDIR} have
  461. ## @description the exec bit set as a pre-exec step.
  462. ## @audience public
  463. ## @stability stable
  464. ## @param filename
  465. ## @param command
  466. ## @param [..]
  467. ## @replaceable no
  468. ## @returns $?
  469. function echo_and_redirect
  470. {
  471. local logfile=$1
  472. shift
  473. verify_patchdir_still_exists
  474. find "${BASEDIR}" -type d -exec chmod +x {} \;
  475. echo "${*} > ${logfile} 2>&1"
  476. "${@}" > "${logfile}" 2>&1
  477. }
  478. ## @description Print the usage information
  479. ## @audience public
  480. ## @stability stable
  481. ## @replaceable no
  482. function hadoop_usage
  483. {
  484. local -r up=$(echo ${PROJECT_NAME} | tr '[:lower:]' '[:upper:]')
  485. echo "Usage: test-patch.sh [options] patch-file | issue-number | http"
  486. echo
  487. echo "Where:"
  488. echo " patch-file is a local patch file containing the changes to test"
  489. echo " issue-number is a 'Patch Available' JIRA defect number (e.g. '${up}-9902') to test"
  490. echo " http is an HTTP address to download the patch file"
  491. echo
  492. echo "Options:"
  493. echo "--basedir=<dir> The directory to apply the patch to (default current directory)"
  494. echo "--branch=<dir> Forcibly set the branch"
  495. echo "--build-native=<bool> If true, then build native components (default 'true')"
  496. echo "--debug If set, then output some extra stuff to stderr"
  497. echo "--dirty-workspace Allow the local git workspace to have uncommitted changes"
  498. echo "--findbugs-home=<path> Findbugs home directory (default FINDBUGS_HOME environment variable)"
  499. echo "--modulelist=<list> Specify additional modules to test (comma delimited)"
  500. echo "--offline Avoid connecting to the Internet"
  501. echo "--patch-dir=<dir> The directory for working and output files (default '/tmp/${PROJECT_NAME}-test-patch/pid')"
  502. echo "--resetrepo Forcibly clean the repo"
  503. echo "--run-tests Run all relevant tests below the base directory"
  504. echo "--testlist=<list> Specify which subsystem tests to use (comma delimited)"
  505. echo "Shell binary overrides:"
  506. echo "--awk-cmd=<cmd> The 'awk' command to use (default 'awk')"
  507. echo "--diff-cmd=<cmd> The GNU-compatible 'diff' command to use (default 'diff')"
  508. echo "--git-cmd=<cmd> The 'git' command to use (default 'git')"
  509. echo "--grep-cmd=<cmd> The 'grep' command to use (default 'grep')"
  510. echo "--mvn-cmd=<cmd> The 'mvn' command to use (default \${MAVEN_HOME}/bin/mvn, or 'mvn')"
  511. echo "--patch-cmd=<cmd> The 'patch' command to use (default 'patch')"
  512. echo "--ps-cmd=<cmd> The 'ps' command to use (default 'ps')"
  513. echo "--sed-cmd=<cmd> The 'sed' command to use (default 'sed')"
  514. echo
  515. echo "Jenkins-only options:"
  516. echo "--jenkins Run by Jenkins (runs tests and posts results to JIRA)"
  517. echo "--eclipse-home=<path> Eclipse home directory (default ECLIPSE_HOME environment variable)"
  518. echo "--jira-cmd=<cmd> The 'jira' command to use (default 'jira')"
  519. echo "--jira-password=<pw> The password for the 'jira' command"
  520. echo "--wget-cmd=<cmd> The 'wget' command to use (default 'wget')"
  521. }
  522. ## @description Interpret the command line parameters
  523. ## @audience private
  524. ## @stability stable
  525. ## @replaceable no
  526. ## @params $@
  527. ## @return May exit on failure
  528. function parse_args
  529. {
  530. local i
  531. local j
  532. for i in "$@"; do
  533. case ${i} in
  534. --awk-cmd=*)
  535. AWK=${i#*=}
  536. ;;
  537. --basedir=*)
  538. BASEDIR=${i#*=}
  539. ;;
  540. --branch=*)
  541. PATCH_BRANCH=${i#*=}
  542. ;;
  543. --build-native=*)
  544. BUILD_NATIVE=${i#*=}
  545. ;;
  546. --debug)
  547. HADOOP_SHELL_SCRIPT_DEBUG=true
  548. ;;
  549. --diff-cmd=*)
  550. DIFF=${i#*=}
  551. ;;
  552. --dirty-workspace)
  553. DIRTY_WORKSPACE=true
  554. ;;
  555. --eclipse-home=*)
  556. ECLIPSE_HOME=${i#*=}
  557. ;;
  558. --findbugs-home=*)
  559. FINDBUGS_HOME=${i#*=}
  560. ;;
  561. --git-cmd=*)
  562. GIT=${i#*=}
  563. ;;
  564. --grep-cmd=*)
  565. GREP=${i#*=}
  566. ;;
  567. --help|-help|-h|help|--h|--\?|-\?|\?)
  568. hadoop_usage
  569. exit 0
  570. ;;
  571. --java-home)
  572. JAVA_HOME=${i#*=}
  573. ;;
  574. --jenkins)
  575. JENKINS=true
  576. ;;
  577. --jira-cmd=*)
  578. JIRACLI=${i#*=}
  579. ;;
  580. --jira-password=*)
  581. JIRA_PASSWD=${i#*=}
  582. ;;
  583. --modulelist=*)
  584. USER_MODULE_LIST=${i#*=}
  585. USER_MODULE_LIST=${USER_MODULE_LIST//,/ }
  586. hadoop_debug "Manually forcing modules ${USER_MODULE_LIST}"
  587. ;;
  588. --mvn-cmd=*)
  589. MVN=${i#*=}
  590. ;;
  591. --offline)
  592. OFFLINE=true
  593. ;;
  594. --patch-cmd=*)
  595. PATCH=${i#*=}
  596. ;;
  597. --patch-dir=*)
  598. PATCH_DIR=${i#*=}
  599. ;;
  600. --ps-cmd=*)
  601. PS=${i#*=}
  602. ;;
  603. --reexec)
  604. REEXECED=true
  605. start_clock
  606. add_jira_table 0 reexec "dev-support patch detected."
  607. ;;
  608. --resetrepo)
  609. RESETREPO=true
  610. ;;
  611. --run-tests)
  612. RUN_TESTS=true
  613. ;;
  614. --testlist=*)
  615. testlist=${i#*=}
  616. testlist=${testlist//,/ }
  617. for j in ${testlist}; do
  618. hadoop_debug "Manually adding patch test subsystem ${j}"
  619. add_test "${j}"
  620. done
  621. ;;
  622. --wget-cmd=*)
  623. WGET=${i#*=}
  624. ;;
  625. *)
  626. PATCH_OR_ISSUE=${i}
  627. ;;
  628. esac
  629. done
  630. # if we get a relative path, turn it absolute
  631. BASEDIR=$(cd -P -- "${BASEDIR}" >/dev/null && pwd -P)
  632. if [[ ${BUILD_NATIVE} == "true" ]] ; then
  633. NATIVE_PROFILE=-Pnative
  634. REQUIRE_TEST_LIB_HADOOP=-Drequire.test.libhadoop
  635. fi
  636. if [[ -z "${PATCH_OR_ISSUE}" ]]; then
  637. hadoop_usage
  638. exit 1
  639. fi
  640. if [[ ${JENKINS} == "true" ]] ; then
  641. echo "Running in Jenkins mode"
  642. ISSUE=${PATCH_OR_ISSUE}
  643. RESETREPO=true
  644. # shellcheck disable=SC2034
  645. ECLIPSE_PROPERTY="-Declipse.home=${ECLIPSE_HOME}"
  646. else
  647. if [[ ${RESETREPO} == "true" ]] ; then
  648. echo "Running in destructive (--resetrepo) developer mode"
  649. else
  650. echo "Running in developer mode"
  651. fi
  652. JENKINS=false
  653. fi
  654. if [[ ! -d ${PATCH_DIR} ]]; then
  655. mkdir -p "${PATCH_DIR}"
  656. if [[ $? == 0 ]] ; then
  657. echo "${PATCH_DIR} has been created"
  658. else
  659. echo "Unable to create ${PATCH_DIR}"
  660. cleanup_and_exit 1
  661. fi
  662. fi
  663. GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
  664. }
  665. ## @description Locate the pom.xml file for a given directory
  666. ## @audience private
  667. ## @stability stable
  668. ## @replaceable no
  669. ## @return directory containing the pom.xml
  670. function find_pom_dir
  671. {
  672. local dir
  673. dir=$(dirname "$1")
  674. hadoop_debug "Find pom dir for: ${dir}"
  675. while builtin true; do
  676. if [[ -f "${dir}/pom.xml" ]];then
  677. echo "${dir}"
  678. hadoop_debug "Found: ${dir}"
  679. return
  680. else
  681. dir=$(dirname "${dir}")
  682. fi
  683. done
  684. }
  685. ## @description List of files that ${PATCH_DIR}/patch modifies
  686. ## @audience private
  687. ## @stability stable
  688. ## @replaceable no
  689. ## @return None; sets ${CHANGED_FILES}
  690. function find_changed_files
  691. {
  692. # get a list of all of the files that have been changed,
  693. # except for /dev/null (which would be present for new files).
  694. # Additionally, remove any a/ b/ patterns at the front
  695. # of the patch filenames and any revision info at the end
  696. # shellcheck disable=SC2016
  697. CHANGED_FILES=$(${GREP} -E '^(\+\+\+|---) ' "${PATCH_DIR}/patch" \
  698. | ${SED} \
  699. -e 's,^....,,' \
  700. -e 's,^[ab]/,,' \
  701. | ${GREP} -v /dev/null \
  702. | ${AWK} '{print $1}' \
  703. | sort -u)
  704. }
  705. ## @description Find the modules of the maven build that ${PATCH_DIR}/patch modifies
  706. ## @audience private
  707. ## @stability stable
  708. ## @replaceable no
  709. ## @return None; sets ${CHANGED_MODULES}
  710. function find_changed_modules
  711. {
  712. # Come up with a list of changed files into ${TMP}
  713. local pomdirs
  714. local module
  715. local pommods
  716. # Now find all the modules that were changed
  717. for file in ${CHANGED_FILES}; do
  718. #shellcheck disable=SC2086
  719. pomdirs="${pomdirs} $(find_pom_dir ${file})"
  720. done
  721. # Filter out modules without code
  722. for module in ${pomdirs}; do
  723. ${GREP} "<packaging>pom</packaging>" "${module}/pom.xml" > /dev/null
  724. if [[ "$?" != 0 ]]; then
  725. pommods="${pommods} ${module}"
  726. fi
  727. done
  728. #shellcheck disable=SC2086
  729. CHANGED_MODULES=$(echo ${pommods} ${USER_MODULE_LIST} | tr ' ' '\n' | sort -u)
  730. }
  731. ## @description git checkout the appropriate branch to test. Additionally, this calls
  732. ## @description 'determine_issue' and 'determine_branch' based upon the context provided
  733. ## @description in ${PATCH_DIR} and in git after checkout.
  734. ## @audience private
  735. ## @stability stable
  736. ## @replaceable no
  737. ## @return 0 on success. May exit on failure.
  738. function git_checkout
  739. {
  740. local currentbranch
  741. big_console_header "Confirming git environment"
  742. if [[ ${RESETREPO} == "true" ]] ; then
  743. cd "${BASEDIR}"
  744. ${GIT} reset --hard
  745. if [[ $? != 0 ]]; then
  746. hadoop_error "ERROR: git reset is failing"
  747. cleanup_and_exit 1
  748. fi
  749. ${GIT} clean -xdf
  750. if [[ $? != 0 ]]; then
  751. hadoop_error "ERROR: git clean is failing"
  752. cleanup_and_exit 1
  753. fi
  754. ${GIT} checkout --force trunk
  755. if [[ $? != 0 ]]; then
  756. hadoop_error "ERROR: git checkout --force trunk is failing"
  757. cleanup_and_exit 1
  758. fi
  759. determine_branch
  760. if [[ ${PATCH_BRANCH} =~ ^git ]]; then
  761. PATCH_BRANCH=$(echo "${PATCH_BRANCH}" | cut -dt -f2)
  762. fi
  763. # we need to explicitly fetch in case the
  764. # git ref hasn't been brought in tree yet
  765. if [[ ${OFFLINE} == false ]]; then
  766. ${GIT} pull --rebase
  767. if [[ $? != 0 ]]; then
  768. hadoop_error "ERROR: git pull is failing"
  769. cleanup_and_exit 1
  770. fi
  771. fi
  772. # forcibly checkout this branch or git ref
  773. ${GIT} checkout --force "${PATCH_BRANCH}"
  774. if [[ $? != 0 ]]; then
  775. hadoop_error "ERROR: git checkout ${PATCH_BRANCH} is failing"
  776. cleanup_and_exit 1
  777. fi
  778. # we need to explicitly fetch in case the
  779. # git ref hasn't been brought in tree yet
  780. if [[ ${OFFLINE} == false ]]; then
  781. ${GIT} pull --rebase
  782. if [[ $? != 0 ]]; then
  783. hadoop_error "ERROR: git pull is failing"
  784. cleanup_and_exit 1
  785. fi
  786. fi
  787. else
  788. cd "${BASEDIR}"
  789. if [[ ! -d .git ]]; then
  790. hadoop_error "ERROR: ${BASEDIR} is not a git repo."
  791. cleanup_and_exit 1
  792. fi
  793. status=$(${GIT} status --porcelain)
  794. if [[ "${status}" != "" && -z ${DIRTY_WORKSPACE} ]] ; then
  795. hadoop_error "ERROR: --dirty-workspace option not provided."
  796. hadoop_error "ERROR: can't run in a workspace that contains the following modifications"
  797. hadoop_error "${status}"
  798. cleanup_and_exit 1
  799. fi
  800. determine_branch
  801. if [[ ${PATCH_BRANCH} =~ ^git ]]; then
  802. PATCH_BRANCH=$(echo "${PATCH_BRANCH}" | cut -dt -f2)
  803. fi
  804. currentbranch=$(${GIT} rev-parse --abbrev-ref HEAD)
  805. if [[ "${currentbranch}" != "${PATCH_BRANCH}" ]];then
  806. echo "WARNING: Current git branch is ${currentbranch} but patch is built for ${PATCH_BRANCH}."
  807. echo "WARNING: Continuing anyway..."
  808. PATCH_BRANCH=${currentbranch}
  809. fi
  810. fi
  811. determine_issue
  812. GIT_REVISION=$(${GIT} rev-parse --verify --short HEAD)
  813. # shellcheck disable=SC2034
  814. VERSION=${GIT_REVISION}_${ISSUE}_PATCH-${patchNum}
  815. if [[ "${ISSUE}" == 'Unknown' ]]; then
  816. echo "Testing patch on ${PATCH_BRANCH}."
  817. else
  818. echo "Testing ${ISSUE} patch on ${PATCH_BRANCH}."
  819. fi
  820. add_jira_footer "git revision" "${PATCH_BRANCH} / ${GIT_REVISION}"
  821. return 0
  822. }
  823. ## @description Confirm the source environment is compilable
  824. ## @audience private
  825. ## @stability stable
  826. ## @replaceable no
  827. ## @return 0 on success
  828. ## @return 1 on failure
  829. function precheck_without_patch
  830. {
  831. local -r mypwd=$(pwd)
  832. big_console_header "Pre-patch ${PATCH_BRANCH} Java verification"
  833. start_clock
  834. verify_needed_test javac
  835. if [[ $? == 1 ]]; then
  836. echo "Compiling ${mypwd}"
  837. echo_and_redirect "${PATCH_DIR}/${PATCH_BRANCH}JavacWarnings.txt" "${MVN}" clean test -DskipTests -D${PROJECT_NAME}PatchProcess -Ptest-patch
  838. if [[ $? != 0 ]] ; then
  839. echo "${PATCH_BRANCH} compilation is broken?"
  840. add_jira_table -1 pre-patch "${PATCH_BRANCH} compilation may be broken."
  841. return 1
  842. fi
  843. else
  844. echo "Patch does not appear to need javac tests."
  845. fi
  846. verify_needed_test javadoc
  847. if [[ $? == 1 ]]; then
  848. echo "Javadoc'ing ${mypwd}"
  849. echo_and_redirect "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarnings.txt" "${MVN}" clean test javadoc:javadoc -DskipTests -Pdocs -D${PROJECT_NAME}PatchProcess
  850. if [[ $? != 0 ]] ; then
  851. echo "Pre-patch ${PATCH_BRANCH} javadoc compilation is broken?"
  852. add_jira_table -1 pre-patch "Pre-patch ${PATCH_BRANCH} JavaDoc compilation may be broken."
  853. return 1
  854. fi
  855. else
  856. echo "Patch does not appear to need javadoc tests."
  857. fi
  858. verify_needed_test site
  859. if [[ $? == 1 ]]; then
  860. echo "site creation for ${mypwd}"
  861. echo_and_redirect "${PATCH_DIR}/${PATCH_BRANCH}SiteWarnings.txt" "${MVN}" clean site site:stage -DskipTests -Dmaven.javadoc.skip=true -D${PROJECT_NAME}PatchProcess
  862. if [[ $? != 0 ]] ; then
  863. echo "Pre-patch ${PATCH_BRANCH} site compilation is broken?"
  864. add_jira_table -1 pre-patch "Pre-patch ${PATCH_BRANCH} site compilation may be broken."
  865. return 1
  866. fi
  867. else
  868. echo "Patch does not appear to need site tests."
  869. fi
  870. add_jira_table 0 pre-patch "Pre-patch ${PATCH_BRANCH} compilation is healthy."
  871. return 0
  872. }
  873. ## @description Confirm the given branch is a member of the list of space
  874. ## @description delimited branches or a git ref
  875. ## @audience private
  876. ## @stability evolving
  877. ## @replaceable no
  878. ## @param branch
  879. ## @param branchlist
  880. ## @return 0 on success
  881. ## @return 1 on failure
  882. function verify_valid_branch
  883. {
  884. local branches=$1
  885. local check=$2
  886. local i
  887. if [[ ${check} =~ ^git ]]; then
  888. ref=$(echo "${check}" | cut -f2 -dt)
  889. count=$(echo "${ref}" | wc -c | tr -d ' ')
  890. if [[ ${count} == 8 || ${count} == 41 ]]; then
  891. return 0
  892. fi
  893. return 1
  894. fi
  895. for i in ${branches}; do
  896. if [[ "${i}" == "${check}" ]]; then
  897. return 0
  898. fi
  899. done
  900. return 1
  901. }
  902. ## @description Try to guess the branch being tested using a variety of heuristics
  903. ## @audience private
  904. ## @stability evolving
  905. ## @replaceable no
  906. ## @return 0 on success, with PATCH_BRANCH updated appropriately
  907. ## @return 1 on failure, with PATCH_BRANCH updated to "trunk"
  908. function determine_branch
  909. {
  910. local allbranches
  911. local patchnamechunk
  912. hadoop_debug "Determine branch"
  913. # something has already set this, so move on
  914. if [[ -n ${PATCH_BRANCH} ]]; then
  915. return
  916. fi
  917. pushd "${BASEDIR}" > /dev/null
  918. # developer mode, existing checkout, whatever
  919. if [[ "${DIRTY_WORKSPACE}" == true ]];then
  920. PATCH_BRANCH=$(${GIT} rev-parse --abbrev-ref HEAD)
  921. echo "dirty workspace mode; applying against existing branch"
  922. return
  923. fi
  924. allbranches=$(${GIT} branch -r | tr -d ' ' | ${SED} -e s,origin/,,g)
  925. for j in "${PATCHURL}" "${PATCH_OR_ISSUE}"; do
  926. hadoop_debug "Determine branch: starting with ${j}"
  927. # shellcheck disable=SC2016
  928. patchnamechunk=$(echo "${j}" | ${AWK} -F/ '{print $NF}')
  929. # ISSUE.branch.##.patch
  930. hadoop_debug "Determine branch: ISSUE.branch.##.patch"
  931. PATCH_BRANCH=$(echo "${patchnamechunk}" | cut -f2 -d. )
  932. verify_valid_branch "${allbranches}" "${PATCH_BRANCH}"
  933. if [[ $? == 0 ]]; then
  934. return
  935. fi
  936. # ISSUE-branch-##.patch
  937. hadoop_debug "Determine branch: ISSUE-branch-##.patch"
  938. PATCH_BRANCH=$(echo "${patchnamechunk}" | cut -f3- -d- | cut -f1,2 -d-)
  939. verify_valid_branch "${allbranches}" "${PATCH_BRANCH}"
  940. if [[ $? == 0 ]]; then
  941. return
  942. fi
  943. # ISSUE-##.patch.branch
  944. hadoop_debug "Determine branch: ISSUE-##.patch.branch"
  945. # shellcheck disable=SC2016
  946. PATCH_BRANCH=$(echo "${patchnamechunk}" | ${AWK} -F. '{print $NF}')
  947. verify_valid_branch "${allbranches}" "${PATCH_BRANCH}"
  948. if [[ $? == 0 ]]; then
  949. return
  950. fi
  951. # ISSUE-branch.##.patch
  952. hadoop_debug "Determine branch: ISSUE-branch.##.patch"
  953. # shellcheck disable=SC2016
  954. PATCH_BRANCH=$(echo "${patchnamechunk}" | cut -f3- -d- | ${AWK} -F. '{print $(NF-2)}' 2>/dev/null)
  955. verify_valid_branch "${allbranches}" "${PATCH_BRANCH}"
  956. if [[ $? == 0 ]]; then
  957. return
  958. fi
  959. done
  960. PATCH_BRANCH=trunk
  961. popd >/dev/null
  962. }
  963. ## @description Try to guess the issue being tested using a variety of heuristics
  964. ## @audience private
  965. ## @stability evolving
  966. ## @replaceable no
  967. ## @return 0 on success, with ISSUE updated appropriately
  968. ## @return 1 on failure, with ISSUE updated to "Unknown"
  969. function determine_issue
  970. {
  971. local patchnamechunk
  972. local maybeissue
  973. hadoop_debug "Determine issue"
  974. # we can shortcut jenkins
  975. if [[ ${JENKINS} == true ]]; then
  976. ISSUE=${PATCH_OR_ISSUE}
  977. fi
  978. # shellcheck disable=SC2016
  979. patchnamechunk=$(echo "${PATCH_OR_ISSUE}" | ${AWK} -F/ '{print $NF}')
  980. maybeissue=$(echo "${patchnamechunk}" | cut -f1,2 -d-)
  981. if [[ ${maybeissue} =~ ${ISSUE_RE} ]]; then
  982. ISSUE=${maybeissue}
  983. return 0
  984. fi
  985. ISSUE="Unknown"
  986. return 1
  987. }
  988. ## @description Add the given test type
  989. ## @audience public
  990. ## @stability stable
  991. ## @replaceable yes
  992. ## @param test
  993. function add_test
  994. {
  995. local testname=$1
  996. hadoop_debug "Testing against ${testname}"
  997. if [[ -z ${NEEDED_TESTS} ]]; then
  998. hadoop_debug "Setting tests to ${testname}"
  999. NEEDED_TESTS=${testname}
  1000. elif [[ ! ${NEEDED_TESTS} =~ ${testname} ]] ; then
  1001. hadoop_debug "Adding ${testname}"
  1002. NEEDED_TESTS="${NEEDED_TESTS} ${testname}"
  1003. fi
  1004. }
  1005. ## @description Verify if a given test was requested
  1006. ## @audience public
  1007. ## @stability stable
  1008. ## @replaceable yes
  1009. ## @param test
  1010. ## @return 1 = yes
  1011. ## @return 0 = no
  1012. function verify_needed_test
  1013. {
  1014. local i=$1
  1015. if [[ ${NEEDED_TESTS} =~ $i ]]; then
  1016. return 1
  1017. fi
  1018. return 0
  1019. }
  1020. ## @description Use some heuristics to determine which long running
  1021. ## @description tests to run
  1022. ## @audience private
  1023. ## @stability stable
  1024. ## @replaceable no
  1025. function determine_needed_tests
  1026. {
  1027. local i
  1028. for i in ${CHANGED_FILES}; do
  1029. if [[ ${i} =~ src/main/webapp ]]; then
  1030. hadoop_debug "tests/webapp: ${i}"
  1031. elif [[ ${i} =~ \.sh
  1032. || ${i} =~ \.cmd
  1033. ]]; then
  1034. hadoop_debug "tests/shell: ${i}"
  1035. elif [[ ${i} =~ \.md$
  1036. || ${i} =~ \.md\.vm$
  1037. || ${i} =~ src/site
  1038. || ${i} =~ src/main/docs
  1039. ]]; then
  1040. hadoop_debug "tests/site: ${i}"
  1041. add_test site
  1042. elif [[ ${i} =~ \.c$
  1043. || ${i} =~ \.cc$
  1044. || ${i} =~ \.h$
  1045. || ${i} =~ \.hh$
  1046. || ${i} =~ \.proto$
  1047. || ${i} =~ src/test
  1048. || ${i} =~ \.cmake$
  1049. || ${i} =~ CMakeLists.txt
  1050. ]]; then
  1051. hadoop_debug "tests/units: ${i}"
  1052. add_test javac
  1053. add_test unit
  1054. elif [[ ${i} =~ pom.xml$
  1055. || ${i} =~ \.java$
  1056. || ${i} =~ src/main
  1057. ]]; then
  1058. hadoop_debug "tests/javadoc+units: ${i}"
  1059. add_test javadoc
  1060. add_test javac
  1061. add_test unit
  1062. fi
  1063. if [[ ${i} =~ \.java$ ]]; then
  1064. add_test findbugs
  1065. fi
  1066. for plugin in ${PLUGINS}; do
  1067. if declare -f ${plugin}_filefilter >/dev/null 2>&1; then
  1068. "${plugin}_filefilter" "${i}"
  1069. fi
  1070. done
  1071. done
  1072. add_jira_footer "Optional Tests" "${NEEDED_TESTS}"
  1073. }
  1074. ## @description Given ${PATCH_ISSUE}, determine what type of patch file is in use, and do the
  1075. ## @description necessary work to place it into ${PATCH_DIR}/patch.
  1076. ## @audience private
  1077. ## @stability evolving
  1078. ## @replaceable no
  1079. ## @return 0 on success
  1080. ## @return 1 on failure, may exit
  1081. function locate_patch
  1082. {
  1083. hadoop_debug "locate patch"
  1084. if [[ -f ${PATCH_OR_ISSUE} ]]; then
  1085. PATCH_FILE="${PATCH_OR_ISSUE}"
  1086. else
  1087. if [[ ${PATCH_OR_ISSUE} =~ ^http ]]; then
  1088. echo "Patch is being downloaded at $(date) from"
  1089. PATCHURL="${PATCH_OR_ISSUE}"
  1090. else
  1091. ${WGET} -q -O "${PATCH_DIR}/jira" "http://issues.apache.org/jira/browse/${PATCH_OR_ISSUE}"
  1092. if [[ $? != 0 ]];then
  1093. hadoop_error "ERROR: Unable to determine what ${PATCH_OR_ISSUE} may reference."
  1094. cleanup_and_exit 1
  1095. fi
  1096. if [[ $(${GREP} -c 'Patch Available' "${PATCH_DIR}/jira") == 0 ]] ; then
  1097. if [[ ${JENKINS} == true ]]; then
  1098. hadoop_error "ERROR: ${PATCH_OR_ISSUE} is not \"Patch Available\"."
  1099. cleanup_and_exit 1
  1100. else
  1101. hadoop_error "WARNING: ${PATCH_OR_ISSUE} is not \"Patch Available\"."
  1102. fi
  1103. fi
  1104. relativePatchURL=$(${GREP} -o '"/jira/secure/attachment/[0-9]*/[^"]*' "${PATCH_DIR}/jira" | ${GREP} -v -e 'htm[l]*$' | sort | tail -1 | ${GREP} -o '/jira/secure/attachment/[0-9]*/[^"]*')
  1105. PATCHURL="http://issues.apache.org${relativePatchURL}"
  1106. if [[ ! ${PATCHURL} =~ \.patch$ ]]; then
  1107. hadoop_error "ERROR: ${PATCHURL} is not a patch file."
  1108. cleanup_and_exit 1
  1109. fi
  1110. patchNum=$(echo "${PATCHURL}" | ${GREP} -o '[0-9]*/' | ${GREP} -o '[0-9]*')
  1111. echo "${ISSUE} patch is being downloaded at $(date) from"
  1112. fi
  1113. echo "${PATCHURL}"
  1114. add_jira_footer "Patch URL" "${PATCHURL}"
  1115. ${WGET} -q -O "${PATCH_DIR}/patch" "${PATCHURL}"
  1116. if [[ $? != 0 ]];then
  1117. hadoop_error "ERROR: ${PATCH_OR_ISSUE} could not be downloaded."
  1118. cleanup_and_exit 1
  1119. fi
  1120. PATCH_FILE="${PATCH_DIR}/patch"
  1121. fi
  1122. if [[ ! -f "${PATCH_DIR}/patch" ]]; then
  1123. cp "${PATCH_FILE}" "${PATCH_DIR}/patch"
  1124. if [[ $? == 0 ]] ; then
  1125. echo "Patch file ${PATCH_FILE} copied to ${PATCH_DIR}"
  1126. else
  1127. hadoop_error "ERROR: Could not copy ${PATCH_FILE} to ${PATCH_DIR}"
  1128. cleanup_and_exit 1
  1129. fi
  1130. fi
  1131. }
  1132. ## @description Given ${PATCH_DIR}/patch, verify the patch is good using ${BINDIR}/smart-apply-patch.sh
  1133. ## @description in dryrun mode.
  1134. ## @audience private
  1135. ## @stability evolving
  1136. ## @replaceable no
  1137. ## @return 0 on success
  1138. ## @return 1 on failure
  1139. function verify_patch_file
  1140. {
  1141. # Before building, check to make sure that the patch is valid
  1142. export PATCH
  1143. "${BINDIR}/smart-apply-patch.sh" "${PATCH_DIR}/patch" dryrun
  1144. if [[ $? != 0 ]] ; then
  1145. echo "PATCH APPLICATION FAILED"
  1146. add_jira_table -1 patch "The patch command could not apply the patch during dryrun."
  1147. return 1
  1148. else
  1149. return 0
  1150. fi
  1151. }
  1152. ## @description Given ${PATCH_DIR}/patch, apply the patch using ${BINDIR}/smart-apply-patch.sh
  1153. ## @audience private
  1154. ## @stability evolving
  1155. ## @replaceable no
  1156. ## @return 0 on success
  1157. ## @return exit on failure
  1158. function apply_patch_file
  1159. {
  1160. big_console_header "Applying patch"
  1161. export PATCH
  1162. "${BINDIR}/smart-apply-patch.sh" "${PATCH_DIR}/patch"
  1163. if [[ $? != 0 ]] ; then
  1164. echo "PATCH APPLICATION FAILED"
  1165. ((RESULT = RESULT + 1))
  1166. add_jira_table -1 patch "The patch command could not apply the patch."
  1167. output_to_console 1
  1168. output_to_jira 1
  1169. cleanup_and_exit 1
  1170. fi
  1171. return 0
  1172. }
  1173. ## @description If this patches actually patches test-patch.sh, then
  1174. ## @description run with the patched version for the test.
  1175. ## @audience private
  1176. ## @stability evolving
  1177. ## @replaceable no
  1178. ## @return none; otherwise relaunches
  1179. function check_reexec
  1180. {
  1181. set +x
  1182. local commentfile=${PATCH_DIR}/tp.${RANDOM}
  1183. if [[ ${REEXECED} == true ]]; then
  1184. big_console_header "Re-exec mode detected. Continuing."
  1185. return
  1186. fi
  1187. if [[ ! ${CHANGED_FILES} =~ dev-support/test-patch
  1188. || ${CHANGED_FILES} =~ dev-support/smart-apply ]] ; then
  1189. return
  1190. fi
  1191. big_console_header "dev-support patch detected"
  1192. printf "\n\nRe-executing against patched versions to test.\n\n"
  1193. apply_patch_file
  1194. if [[ ${JENKINS} == true ]]; then
  1195. rm "${commentfile}" 2>/dev/null
  1196. echo "(!) A patch to test-patch or smart-apply-patch has been detected. " > "${commentfile}"
  1197. echo "Re-executing against the patched versions to perform further tests. " >> "${commentfile}"
  1198. echo "The console is at ${BUILD_URL}console in case of problems." >> "${commentfile}"
  1199. write_to_jira "${commentfile}"
  1200. rm "${commentfile}"
  1201. fi
  1202. cd "${CWD}"
  1203. mkdir -p "${PATCH_DIR}/dev-support-test"
  1204. cp -pr "${BASEDIR}"/dev-support/test-patch* "${PATCH_DIR}/dev-support-test"
  1205. cp -pr "${BASEDIR}"/dev-support/smart-apply* "${PATCH_DIR}/dev-support-test"
  1206. big_console_header "exec'ing test-patch.sh now..."
  1207. exec "${PATCH_DIR}/dev-support-test/test-patch.sh" \
  1208. --reexec \
  1209. --branch ${PATCH_BRANCH} \
  1210. --patch-dir="${PATCH_DIR}" \
  1211. "${USER_PARAMS[@]}"
  1212. }
  1213. ## @description Check the current directory for @author tags
  1214. ## @audience private
  1215. ## @stability evolving
  1216. ## @replaceable no
  1217. ## @return 0 on success
  1218. ## @return 1 on failure
  1219. function check_author
  1220. {
  1221. local authorTags
  1222. big_console_header "Checking there are no @author tags in the patch."
  1223. start_clock
  1224. if [[ ${CHANGED_FILES} =~ dev-support/test-patch ]]; then
  1225. add_jira_table 0 @author "Skipping @author checks as test-patch has been patched."
  1226. return 0
  1227. fi
  1228. authorTags=$("${GREP}" -c -i '@author' "${PATCH_DIR}/patch")
  1229. echo "There appear to be ${authorTags} @author tags in the patch."
  1230. if [[ ${authorTags} != 0 ]] ; then
  1231. add_jira_table -1 @author \
  1232. "The patch appears to contain ${authorTags} @author tags which the Hadoop" \
  1233. " community has agreed to not allow in code contributions."
  1234. return 1
  1235. fi
  1236. add_jira_table +1 @author "The patch does not contain any @author tags."
  1237. return 0
  1238. }
  1239. ## @description Check the patch file for changed/new tests
  1240. ## @audience private
  1241. ## @stability evolving
  1242. ## @replaceable no
  1243. ## @return 0 on success
  1244. ## @return 1 on failure
  1245. function check_modified_unittests
  1246. {
  1247. local testReferences=0
  1248. local i
  1249. verify_needed_test unit
  1250. if [[ $? == 0 ]]; then
  1251. return 0
  1252. fi
  1253. big_console_header "Checking there are new or changed tests in the patch."
  1254. start_clock
  1255. for i in ${CHANGED_FILES}; do
  1256. if [[ ${i} =~ /test/ ]]; then
  1257. ((testReferences=testReferences + 1))
  1258. fi
  1259. done
  1260. echo "There appear to be ${testReferences} test file(s) referenced in the patch."
  1261. if [[ ${testReferences} == 0 ]] ; then
  1262. add_jira_table -1 "tests included" \
  1263. "The patch doesn't appear to include any new or modified tests. " \
  1264. "Please justify why no new tests are needed for this patch." \
  1265. "Also please list what manual steps were performed to verify this patch."
  1266. return 1
  1267. fi
  1268. add_jira_table +1 "tests included" \
  1269. "The patch appears to include ${testReferences} new or modified test files."
  1270. return 0
  1271. }
  1272. ## @description Helper for check_javadoc
  1273. ## @audience private
  1274. ## @stability evolving
  1275. ## @replaceable no
  1276. ## @return 0 on success
  1277. ## @return 1 on failure
  1278. function count_javadoc_warns
  1279. {
  1280. local warningfile=$1
  1281. #shellcheck disable=SC2016,SC2046
  1282. return $(${EGREP} "^[0-9]+ warnings$" "${warningfile}" | ${AWK} '{sum+=$1} END {print sum}')
  1283. }
  1284. ## @description Count and compare the number of JavaDoc warnings pre- and post- patch
  1285. ## @audience private
  1286. ## @stability evolving
  1287. ## @replaceable no
  1288. ## @return 0 on success
  1289. ## @return 1 on failure
  1290. function check_javadoc
  1291. {
  1292. local numBranchJavadocWarnings
  1293. local numPatchJavadocWarnings
  1294. verify_needed_test javadoc
  1295. if [[ $? == 0 ]]; then
  1296. echo "This patch does not appear to need javadoc checks."
  1297. return 0
  1298. fi
  1299. big_console_header "Determining number of patched javadoc warnings"
  1300. start_clock
  1301. if [[ -d hadoop-project ]]; then
  1302. (cd hadoop-project; "${MVN}" install > /dev/null 2>&1)
  1303. fi
  1304. if [[ -d hadoop-common-project/hadoop-annotations ]]; then
  1305. (cd hadoop-common-project/hadoop-annotations; "${MVN}" install > /dev/null 2>&1)
  1306. fi
  1307. echo_and_redirect "${PATCH_DIR}/patchJavadocWarnings.txt" "${MVN}" clean test javadoc:javadoc -DskipTests -Pdocs -D${PROJECT_NAME}PatchProcess
  1308. count_javadoc_warns "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarnings.txt"
  1309. numBranchJavadocWarnings=$?
  1310. count_javadoc_warns "${PATCH_DIR}/patchJavadocWarnings.txt"
  1311. numPatchJavadocWarnings=$?
  1312. echo "There appear to be ${numBranchJavadocWarnings} javadoc warnings before the patch and ${numPatchJavadocWarnings} javadoc warnings after applying the patch."
  1313. if [[ ${numBranchJavadocWarnings} != "" && ${numPatchJavadocWarnings} != "" ]] ; then
  1314. if [[ ${numPatchJavadocWarnings} -gt ${numBranchJavadocWarnings} ]] ; then
  1315. ${GREP} -i warning "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarnings.txt" > "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarningsFiltered.txt"
  1316. ${GREP} -i warning "${PATCH_DIR}/patchJavadocWarnings.txt" > "${PATCH_DIR}/patchJavadocWarningsFiltered.txt"
  1317. ${DIFF} -u "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarningsFiltered.txt" \
  1318. "${PATCH_DIR}/patchJavadocWarningsFiltered.txt" \
  1319. > "${PATCH_DIR}/diffJavadocWarnings.txt"
  1320. rm -f "${PATCH_DIR}/${PATCH_BRANCH}JavadocWarningsFiltered.txt" "${PATCH_DIR}/patchJavadocWarningsFiltered.txt"
  1321. add_jira_table -1 javadoc "The applied patch generated "\
  1322. "$((numPatchJavadocWarnings-numBranchJavadocWarnings))" \
  1323. " additional warning messages."
  1324. add_jira_footer javadoc "@@BASE@@/diffJavadocWarnings.txt"
  1325. return 1
  1326. fi
  1327. fi
  1328. add_jira_table +1 javadoc "There were no new javadoc warning messages."
  1329. return 0
  1330. }
  1331. ## @description Make sure site still compiles
  1332. ## @audience private
  1333. ## @stability evolving
  1334. ## @replaceable no
  1335. ## @return 0 on success
  1336. ## @return 1 on failure
  1337. function check_site
  1338. {
  1339. local -r mypwd=$(pwd)
  1340. verify_needed_test site
  1341. if [[ $? == 0 ]]; then
  1342. echo "This patch does not appear to need site checks."
  1343. return 0
  1344. fi
  1345. big_console_header "Determining if patched site still builds"
  1346. start_clock
  1347. echo "site creation for ${mypwd}"
  1348. echo_and_redirect "${PATCH_DIR}/patchSiteWarnings.txt" "${MVN}" clean site site:stage -DskipTests -Dmaven.javadoc.skip=true -D${PROJECT_NAME}PatchProcess
  1349. if [[ $? != 0 ]] ; then
  1350. echo "Site compilation is broken"
  1351. add_jira_table -1 site "Site compilation is broken."
  1352. add_jira_footer site "@@BASE@@/patchSiteWarnings.txt"
  1353. return 1
  1354. fi
  1355. add_jira_table +1 site "Site still builds."
  1356. return 0
  1357. }
  1358. ## @description Helper for check_javac
  1359. ## @audience private
  1360. ## @stability evolving
  1361. ## @replaceable no
  1362. ## @return 0 on success
  1363. ## @return 1 on failure
  1364. function count_javac_warns
  1365. {
  1366. local warningfile=$1
  1367. #shellcheck disable=SC2016,SC2046
  1368. return $(${AWK} 'BEGIN {total = 0} {total += 1} END {print total}' "${warningfile}")
  1369. }
  1370. ## @description Count and compare the number of javac warnings pre- and post- patch
  1371. ## @audience private
  1372. ## @stability evolving
  1373. ## @replaceable no
  1374. ## @return 0 on success
  1375. ## @return 1 on failure
  1376. function check_javac
  1377. {
  1378. local branchJavacWarnings
  1379. local patchJavacWarnings
  1380. verify_needed_test javac
  1381. if [[ $? == 0 ]]; then
  1382. echo "This patch does not appear to need javac checks."
  1383. return 0
  1384. fi
  1385. big_console_header "Determining number of patched javac warnings."
  1386. start_clock
  1387. echo_and_redirect "${PATCH_DIR}/patchJavacWarnings.txt" "${MVN}" clean test -DskipTests -D${PROJECT_NAME}PatchProcess ${NATIVE_PROFILE} -Ptest-patch
  1388. if [[ $? != 0 ]] ; then
  1389. add_jira_table -1 javac "The patch appears to cause the build to fail."
  1390. return 2
  1391. fi
  1392. ### Compare ${PATCH_BRANCH} and patch javac warning numbers
  1393. if [[ -f ${PATCH_DIR}/patchJavacWarnings.txt ]] ; then
  1394. ${GREP} '\[WARNING\]' "${PATCH_DIR}/${PATCH_BRANCH}JavacWarnings.txt" > "${PATCH_DIR}/filtered${PATCH_BRANCH}JavacWarnings.txt"
  1395. ${GREP} '\[WARNING\]' "${PATCH_DIR}/patchJavacWarnings.txt" > "${PATCH_DIR}/filteredPatchJavacWarnings.txt"
  1396. count_javac_warns "${PATCH_DIR}/filtered${PATCH_BRANCH}JavacWarnings.txt"
  1397. branchJavacWarnings=$?
  1398. count_javac_warns "${PATCH_DIR}/filteredPatchJavacWarnings.txt"
  1399. patchJavacWarnings=$?
  1400. echo "There appear to be ${branchJavacWarnings} javac compiler warnings before the patch and ${patchJavacWarnings} javac compiler warnings after applying the patch."
  1401. if [[ ${patchJavacWarnings} != "" && ${branchJavacWarnings} != "" ]] ; then
  1402. if [[ ${patchJavacWarnings} -gt ${branchJavacWarnings} ]] ; then
  1403. ${DIFF} "${PATCH_DIR}/filtered${PATCH_BRANCH}JavacWarnings.txt" \
  1404. "${PATCH_DIR}/filteredPatchJavacWarnings.txt" \
  1405. > "${PATCH_DIR}/diffJavacWarnings.txt"
  1406. add_jira_table -1 javac "The applied patch generated "\
  1407. "$((patchJavacWarnings-${PATCH_BRANCH}JavacWarnings))" \
  1408. " additional warning messages."
  1409. add_jira_footer javac "@@BASE@@/diffJavacWarnings.txt"
  1410. return 1
  1411. fi
  1412. fi
  1413. fi
  1414. add_jira_table +1 javac "There were no new javac warning messages."
  1415. return 0
  1416. }
  1417. ## @description Verify all files have an Apache License
  1418. ## @audience private
  1419. ## @stability evolving
  1420. ## @replaceable no
  1421. ## @return 0 on success
  1422. ## @return 1 on failure
  1423. function check_apachelicense
  1424. {
  1425. big_console_header "Determining number of patched release audit warnings."
  1426. start_clock
  1427. echo_and_redirect "${PATCH_DIR}/patchReleaseAuditOutput.txt" "${MVN}" apache-rat:check -D${PROJECT_NAME}PatchProcess
  1428. #shellcheck disable=SC2038
  1429. find "${BASEDIR}" -name rat.txt | xargs cat > "${PATCH_DIR}/patchReleaseAuditWarnings.txt"
  1430. ### Compare ${PATCH_BRANCH} and patch release audit warning numbers
  1431. if [[ -f ${PATCH_DIR}/patchReleaseAuditWarnings.txt ]] ; then
  1432. patchReleaseAuditWarnings=$("${GREP}" -c '\!?????' "${PATCH_DIR}/patchReleaseAuditWarnings.txt")
  1433. echo ""
  1434. echo ""
  1435. echo "There appear to be ${patchReleaseAuditWarnings} release audit warnings after applying the patch."
  1436. if [[ ${patchReleaseAuditWarnings} != "" ]] ; then
  1437. if [[ ${patchReleaseAuditWarnings} -gt 0 ]] ; then
  1438. add_jira_table -1 "release audit" "The applied patch generated ${patchReleaseAuditWarnings} release audit warnings."
  1439. ${GREP} '\!?????' "${PATCH_DIR}/patchReleaseAuditWarnings.txt" \
  1440. > "${PATCH_DIR}/patchReleaseAuditProblems.txt"
  1441. echo "Lines that start with ????? in the release audit "\
  1442. "report indicate files that do not have an Apache license header." \
  1443. >> "${PATCH_DIR}/patchReleaseAuditProblems.txt"
  1444. add_jira_footer "Release Audit" "@@BASE@@/patchReleaseAuditProblems.txt"
  1445. return 1
  1446. fi
  1447. fi
  1448. fi
  1449. add_jira_table 1 "release audit" "The applied patch does not increase the total number of release audit warnings."
  1450. return 0
  1451. }
  1452. ## @description Verify mvn install works
  1453. ## @audience private
  1454. ## @stability evolving
  1455. ## @replaceable no
  1456. ## @return 0 on success
  1457. ## @return 1 on failure
  1458. function check_mvn_install
  1459. {
  1460. local retval
  1461. verify_needed_test javadoc
  1462. retval=$?
  1463. verify_needed_test javac
  1464. ((retval = retval + $? ))
  1465. if [[ ${retval} == 0 ]]; then
  1466. echo "This patch does not appear to need mvn install checks."
  1467. return 0
  1468. fi
  1469. big_console_header "Installing all of the jars"
  1470. start_clock
  1471. echo_and_redirect "${PATCH_DIR}/jarinstall.txt" "${MVN}" install -Dmaven.javadoc.skip=true -DskipTests -D${PROJECT_NAME}PatchProcess
  1472. retval=$?
  1473. if [[ ${retval} != 0 ]]; then
  1474. add_jira_table -1 install "The patch causes mvn install to fail."
  1475. else
  1476. add_jira_table +1 install "mvn install still works."
  1477. fi
  1478. return ${retval}
  1479. }
  1480. ## @description Verify patch does not trigger any findbugs warnings
  1481. ## @audience private
  1482. ## @stability evolving
  1483. ## @replaceable no
  1484. ## @return 0 on success
  1485. ## @return 1 on failure
  1486. function check_findbugs
  1487. {
  1488. local findbugs_version
  1489. local modules=${CHANGED_MODULES}
  1490. local rc=0
  1491. local module_suffix
  1492. local findbugsWarnings=0
  1493. local relative_file
  1494. local newFindbugsWarnings
  1495. local findbugsWarnings
  1496. local line
  1497. local firstpart
  1498. local secondpart
  1499. big_console_header "Determining number of patched Findbugs warnings."
  1500. verify_needed_test findbugs
  1501. if [[ $? == 0 ]]; then
  1502. echo "Patch does not touch any java files. Skipping findbugs."
  1503. return 0
  1504. fi
  1505. start_clock
  1506. if [[ ! -e "${FINDBUGS_HOME}/bin/findbugs" ]]; then
  1507. printf "\n\n%s is not executable.\n\n" "${FINDBUGS_HOME}/bin/findbugs"
  1508. add_jira_table -1 findbugs "Findbugs is not installed."
  1509. return 1
  1510. fi
  1511. findbugs_version=$("${FINDBUGS_HOME}/bin/findbugs" -version)
  1512. for module in ${modules}
  1513. do
  1514. pushd "${module}" >/dev/null
  1515. echo " Running findbugs in ${module}"
  1516. module_suffix=$(basename "${module}")
  1517. echo_and_redirect "${PATCH_DIR}/patchFindBugsOutput${module_suffix}.txt" "${MVN}" clean test findbugs:findbugs -DskipTests -D${PROJECT_NAME}PatchProcess \
  1518. < /dev/null
  1519. (( rc = rc + $? ))
  1520. popd >/dev/null
  1521. done
  1522. if [[ ${rc} -ne 0 ]]; then
  1523. add_jira_table -1 findbugs "The patch appears to cause Findbugs (version ${findbugs_version}) to fail."
  1524. return 1
  1525. fi
  1526. while read file
  1527. do
  1528. relative_file=${file#${BASEDIR}/} # strip leading ${BASEDIR} prefix
  1529. if [[ ${relative_file} != "target/findbugsXml.xml" ]]; then
  1530. module_suffix=${relative_file%/target/findbugsXml.xml} # strip trailing path
  1531. module_suffix=$(basename "${module_suffix}")
  1532. fi
  1533. cp "${file}" "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml"
  1534. "${FINDBUGS_HOME}/bin/setBugDatabaseInfo" -timestamp "01/01/2000" \
  1535. "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
  1536. "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml"
  1537. #shellcheck disable=SC2016
  1538. newFindbugsWarnings=$("${FINDBUGS_HOME}/bin/filterBugs" \
  1539. -first "01/01/2000" "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
  1540. "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.xml" \
  1541. | ${AWK} '{print $1}')
  1542. echo "Found $newFindbugsWarnings Findbugs warnings ($file)"
  1543. findbugsWarnings=$((findbugsWarnings+newFindbugsWarnings))
  1544. "${FINDBUGS_HOME}/bin/convertXmlToText" -html \
  1545. "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.xml" \
  1546. "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.html"
  1547. if [[ ${newFindbugsWarnings} -gt 0 ]] ; then
  1548. populate_test_table FindBugs "module:${module_suffix}"
  1549. while read line; do
  1550. firstpart=$(echo "${line}" | cut -f2 -d:)
  1551. secondpart=$(echo "${line}" | cut -f9- -d' ')
  1552. add_jira_test_table "" "${firstpart}:${secondpart}"
  1553. done < <("${FINDBUGS_HOME}/bin/convertXmlToText" \
  1554. "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.xml")
  1555. add_jira_footer "Findbugs warnings" "@@BASE@@/newPatchFindbugsWarnings${module_suffix}.html"
  1556. fi
  1557. done < <(find "${BASEDIR}" -name findbugsXml.xml)
  1558. if [[ ${findbugsWarnings} -gt 0 ]] ; then
  1559. add_jira_table -1 findbugs "The patch appears to introduce ${findbugsWarnings} new Findbugs (version ${findbugs_version}) warnings."
  1560. return 1
  1561. fi
  1562. add_jira_table +1 findbugs "The patch does not introduce any new Findbugs (version ${findbugs_version}) warnings."
  1563. return 0
  1564. }
  1565. ## @description Make sure Maven's eclipse generation works.
  1566. ## @audience private
  1567. ## @stability evolving
  1568. ## @replaceable no
  1569. ## @return 0 on success
  1570. ## @return 1 on failure
  1571. function check_mvn_eclipse
  1572. {
  1573. big_console_header "Running mvn eclipse:eclipse."
  1574. verify_needed_test javac
  1575. if [[ $? == 0 ]]; then
  1576. echo "Patch does not touch any java files. Skipping mvn eclipse:eclipse"
  1577. return 0
  1578. fi
  1579. start_clock
  1580. echo_and_redirect "${PATCH_DIR}/patchEclipseOutput.txt" "${MVN}" eclipse:eclipse -D${PROJECT_NAME}PatchProcess
  1581. if [[ $? != 0 ]] ; then
  1582. add_jira_table -1 eclipse:eclipse "The patch failed to build with eclipse:eclipse."
  1583. return 1
  1584. fi
  1585. add_jira_table +1 eclipse:eclipse "The patch built with eclipse:eclipse."
  1586. return 0
  1587. }
  1588. ## @description Utility to push many tests into the failure list
  1589. ## @audience private
  1590. ## @stability evolving
  1591. ## @replaceable no
  1592. ## @param testdesc
  1593. ## @param testlist
  1594. function populate_test_table
  1595. {
  1596. local reason=$1
  1597. shift
  1598. local first=""
  1599. local i
  1600. for i in "$@"; do
  1601. if [[ -z "${first}" ]]; then
  1602. add_jira_test_table "${reason}" "${i}"
  1603. first="${reason}"
  1604. else
  1605. add_jira_test_table " " "${i}"
  1606. fi
  1607. done
  1608. }
  1609. ## @description Run and verify the output of the appropriate unit tests
  1610. ## @audience private
  1611. ## @stability evolving
  1612. ## @replaceable no
  1613. ## @return 0 on success
  1614. ## @return 1 on failure
  1615. function check_unittests
  1616. {
  1617. verify_needed_test unit
  1618. if [[ $? == 0 ]]; then
  1619. echo "Existing unit tests do not test patched files. Skipping."
  1620. return 0
  1621. fi
  1622. big_console_header "Running unit tests"
  1623. start_clock
  1624. local failed_tests=""
  1625. local modules=${CHANGED_MODULES}
  1626. local building_common=0
  1627. local hdfs_modules
  1628. local ordered_modules=""
  1629. local failed_test_builds=""
  1630. local test_timeouts=""
  1631. local test_logfile
  1632. local test_build_result
  1633. local module_test_timeouts=""
  1634. local result
  1635. local totalresult=0
  1636. local module_prefix
  1637. #
  1638. # If we are building hadoop-hdfs-project, we must build the native component
  1639. # of hadoop-common-project first. In order to accomplish this, we move the
  1640. # hadoop-hdfs subprojects to the end of the list so that common will come
  1641. # first.
  1642. #
  1643. # Of course, we may not be building hadoop-common at all-- in this case, we
  1644. # explicitly insert a mvn compile -Pnative of common, to ensure that the
  1645. # native libraries show up where we need them.
  1646. #
  1647. for module in ${modules}; do
  1648. if [[ ${module} == hadoop-hdfs-project* ]]; then
  1649. hdfs_modules="${hdfs_modules} ${module}"
  1650. elif [[ ${module} == hadoop-common-project* ]]; then
  1651. ordered_modules="${ordered_modules} ${module}"
  1652. building_common=1
  1653. else
  1654. ordered_modules="${ordered_modules} ${module}"
  1655. fi
  1656. done
  1657. if [[ -n "${hdfs_modules}" ]]; then
  1658. ordered_modules="${ordered_modules} ${hdfs_modules}"
  1659. if [[ ${building_common} -eq 0 ]]; then
  1660. echo " Building hadoop-common with -Pnative in order to provide libhadoop.so to the hadoop-hdfs unit tests."
  1661. echo_and_redirect "${PATCH_DIR}/testrun_native.txt" "${MVN}" compile ${NATIVE_PROFILE} "-D${PROJECT_NAME}PatchProcess"
  1662. if [[ $? != 0 ]]; then
  1663. add_jira_table -1 "native" "Failed to build the native portion " \
  1664. "of hadoop-common prior to running the unit tests in ${ordered_modules}"
  1665. return 1
  1666. else
  1667. add_jira_table +1 "native" "Pre-build of native portion"
  1668. fi
  1669. fi
  1670. fi
  1671. for module in ${ordered_modules}; do
  1672. result=0
  1673. start_clock
  1674. pushd "${module}" >/dev/null
  1675. module_suffix=$(basename "${module}")
  1676. module_prefix=$(echo "${module}" | cut -f2 -d- )
  1677. test_logfile=${PATCH_DIR}/testrun_${module_suffix}.txt
  1678. echo " Running tests in ${module_suffix}"
  1679. echo_and_redirect "${test_logfile}" "${MVN}" clean install -fae ${NATIVE_PROFILE} ${REQUIRE_TEST_LIB_HADOOP} -D${PROJECT_NAME}PatchProcess
  1680. test_build_result=$?
  1681. add_jira_footer "${module_suffix} test log" "@@BASE@@/testrun_${module_suffix}.txt"
  1682. # shellcheck disable=2016
  1683. module_test_timeouts=$(${AWK} '/^Running / { if (last) { print last } last=$2 } /^Tests run: / { last="" }' "${test_logfile}")
  1684. if [[ -n "${module_test_timeouts}" ]] ; then
  1685. test_timeouts="${test_timeouts} ${module_test_timeouts}"
  1686. result=1
  1687. fi
  1688. #shellcheck disable=SC2026,SC2038,SC2016
  1689. module_failed_tests=$(find . -name 'TEST*.xml'\
  1690. | xargs "${GREP}" -l -E "<failure|<error"\
  1691. | ${AWK} -F/ '{sub("TEST-org.apache.",""); sub(".xml",""); print $NF}')
  1692. if [[ -n "${module_failed_tests}" ]] ; then
  1693. failed_tests="${failed_tests} ${module_failed_tests}"
  1694. result=1
  1695. fi
  1696. if [[ ${test_build_result} != 0 && -z "${module_failed_tests}" && -z "${module_test_timeouts}" ]] ; then
  1697. failed_test_builds="${failed_test_builds} ${module_suffix}"
  1698. result=1
  1699. fi
  1700. popd >/dev/null
  1701. if [[ $result == 1 ]]; then
  1702. add_jira_table -1 "${module_prefix} tests" "Tests failed in ${module_suffix}."
  1703. else
  1704. add_jira_table +1 "${module_prefix} tests" "Tests passed in ${module_suffix}."
  1705. fi
  1706. ((totalresult = totalresult + result))
  1707. done
  1708. if [[ -n "${failed_tests}" ]] ; then
  1709. # shellcheck disable=SC2086
  1710. populate_test_table "Failed unit tests" ${failed_tests}
  1711. fi
  1712. if [[ -n "${test_timeouts}" ]] ; then
  1713. # shellcheck disable=SC2086
  1714. populate_test_table "Timed out tests" ${test_timeouts}
  1715. fi
  1716. if [[ -n "${failed_test_builds}" ]] ; then
  1717. # shellcheck disable=SC2086
  1718. populate_test_table "Failed build" ${failed_test_builds}
  1719. fi
  1720. if [[ ${JENKINS} == true ]]; then
  1721. add_jira_footer "Test Results" "${BUILD_URL}testReport/"
  1722. fi
  1723. if [[ ${totalresult} -gt 0 ]]; then
  1724. return 1
  1725. else
  1726. return 0
  1727. fi
  1728. }
  1729. ## @description Print out the finished details on the console
  1730. ## @audience private
  1731. ## @stability evolving
  1732. ## @replaceable no
  1733. ## @param runresult
  1734. ## @return 0 on success
  1735. ## @return 1 on failure
  1736. function output_to_console
  1737. {
  1738. local result=$1
  1739. shift
  1740. local i
  1741. local ourstring
  1742. local vote
  1743. local subs
  1744. local ela
  1745. local comment
  1746. local commentfile1="${PATCH_DIR}/comment.1"
  1747. local commentfile2="${PATCH_DIR}/comment.2"
  1748. local normaltop
  1749. local line
  1750. local seccoladj=0
  1751. local spcfx=${PATCH_DIR}/spcl.txt
  1752. if [[ ${result} == 0 ]]; then
  1753. if [[ ${JENKINS} == false ]]; then
  1754. {
  1755. printf "IF9fX19fX19fX18gCjwgU3VjY2VzcyEgPgogLS0tLS0tLS0tLSAKIFwgICAg";
  1756. printf "IC9cICBfX18gIC9cCiAgXCAgIC8vIFwvICAgXC8gXFwKICAgICAoKCAgICBP";
  1757. printf "IE8gICAgKSkKICAgICAgXFwgLyAgICAgXCAvLwogICAgICAgXC8gIHwgfCAg";
  1758. printf "XC8gCiAgICAgICAgfCAgfCB8ICB8ICAKICAgICAgICB8ICB8IHwgIHwgIAog";
  1759. printf "ICAgICAgIHwgICBvICAgfCAgCiAgICAgICAgfCB8ICAgfCB8ICAKICAgICAg";
  1760. printf "ICB8bXwgICB8bXwgIAo"
  1761. } > "${spcfx}"
  1762. fi
  1763. printf "\n\n+1 overall\n\n"
  1764. else
  1765. if [[ ${JENKINS} == false ]]; then
  1766. {
  1767. printf "IF9fX19fICAgICBfIF8gICAgICAgICAgICAgICAgXyAKfCAgX19ffF8gXyhf";
  1768. printf "KSB8XyAgIF8gXyBfXyBfX198IHwKfCB8XyAvIF9gIHwgfCB8IHwgfCB8ICdf";
  1769. printf "Xy8gXyBcIHwKfCAgX3wgKF98IHwgfCB8IHxffCB8IHwgfCAgX18vX3wKfF98";
  1770. printf "ICBcX18sX3xffF98XF9fLF98X3wgIFxfX18oXykKICAgICAgICAgICAgICAg";
  1771. printf "ICAgICAgICAgICAgICAgICAK"
  1772. } > "${spcfx}"
  1773. fi
  1774. printf "\n\n-1 overall\n\n"
  1775. fi
  1776. if [[ -f ${spcfx} ]]; then
  1777. if which base64 >/dev/null 2>&1; then
  1778. base64 --decode "${spcfx}" 2>/dev/null
  1779. elif which openssl >/dev/null 2>&1; then
  1780. openssl enc -A -d -base64 -in "${spcfx}" 2>/dev/null
  1781. fi
  1782. echo
  1783. echo
  1784. rm "${spcfx}"
  1785. fi
  1786. seccoladj=$(findlargest 2 "${JIRA_COMMENT_TABLE[@]}")
  1787. if [[ ${seccoladj} -lt 10 ]]; then
  1788. seccoladj=10
  1789. fi
  1790. seccoladj=$((seccoladj + 2 ))
  1791. i=0
  1792. until [[ $i -eq ${#JIRA_HEADER[@]} ]]; do
  1793. printf "%s\n" "${JIRA_HEADER[${i}]}"
  1794. ((i=i+1))
  1795. done
  1796. printf "| %s | %*s | %s | %s\n" "Vote" ${seccoladj} Subsystem Runtime "Comment"
  1797. echo "============================================================================"
  1798. i=0
  1799. until [[ $i -eq ${#JIRA_COMMENT_TABLE[@]} ]]; do
  1800. ourstring=$(echo "${JIRA_COMMENT_TABLE[${i}]}" | tr -s ' ')
  1801. vote=$(echo "${ourstring}" | cut -f2 -d\|)
  1802. vote=$(colorstripper "${vote}")
  1803. subs=$(echo "${ourstring}" | cut -f3 -d\|)
  1804. ela=$(echo "${ourstring}" | cut -f4 -d\|)
  1805. comment=$(echo "${ourstring}" | cut -f5 -d\|)
  1806. echo "${comment}" | fold -s -w $((78-seccoladj-22)) > "${commentfile1}"
  1807. normaltop=$(head -1 "${commentfile1}")
  1808. ${SED} -e '1d' "${commentfile1}" > "${commentfile2}"
  1809. printf "| %4s | %*s | %-10s |%-s\n" "${vote}" ${seccoladj} \
  1810. "${subs}" "${ela}" "${normaltop}"
  1811. while read line; do
  1812. printf "| | %*s | | %-s\n" ${seccoladj} " " "${line}"
  1813. done < "${commentfile2}"
  1814. ((i=i+1))
  1815. rm "${commentfile2}" "${commentfile1}" 2>/dev/null
  1816. done
  1817. if [[ ${#JIRA_TEST_TABLE[@]} -gt 0 ]]; then
  1818. seccoladj=$(findlargest 1 "${JIRA_TEST_TABLE[@]}")
  1819. printf "\n\n%*s | Tests\n" "${seccoladj}" "Reason"
  1820. i=0
  1821. until [[ $i -eq ${#JIRA_TEST_TABLE[@]} ]]; do
  1822. ourstring=$(echo "${JIRA_TEST_TABLE[${i}]}" | tr -s ' ')
  1823. vote=$(echo "${ourstring}" | cut -f2 -d\|)
  1824. subs=$(echo "${ourstring}" | cut -f3 -d\|)
  1825. printf "%*s | %s\n" "${seccoladj}" "${vote}" "${subs}"
  1826. ((i=i+1))
  1827. done
  1828. fi
  1829. printf "\n\n|| Subsystem || Report/Notes ||\n"
  1830. echo "============================================================================"
  1831. i=0
  1832. until [[ $i -eq ${#JIRA_FOOTER_TABLE[@]} ]]; do
  1833. comment=$(echo "${JIRA_FOOTER_TABLE[${i}]}" |
  1834. ${SED} -e "s,@@BASE@@,${PATCH_DIR},g")
  1835. printf "%s\n" "${comment}"
  1836. ((i=i+1))
  1837. done
  1838. }
  1839. ## @description Print out the finished details to the JIRA issue
  1840. ## @audience private
  1841. ## @stability evolving
  1842. ## @replaceable no
  1843. ## @param runresult
  1844. function output_to_jira
  1845. {
  1846. local result=$1
  1847. local i
  1848. local commentfile=${PATCH_DIR}/commentfile
  1849. local comment
  1850. rm "${commentfile}" 2>/dev/null
  1851. if [[ ${JENKINS} != "true" ]] ; then
  1852. return 0
  1853. fi
  1854. big_console_header "Adding comment to JIRA"
  1855. add_jira_footer "Console output" "${BUILD_URL}console"
  1856. if [[ ${result} == 0 ]]; then
  1857. add_jira_header "(/) *{color:green}+1 overall{color}*"
  1858. else
  1859. add_jira_header "(x) *{color:red}-1 overall{color}*"
  1860. fi
  1861. { echo "\\\\" ; echo "\\\\"; } >> "${commentfile}"
  1862. i=0
  1863. until [[ $i -eq ${#JIRA_HEADER[@]} ]]; do
  1864. printf "%s\n" "${JIRA_HEADER[${i}]}" >> "${commentfile}"
  1865. ((i=i+1))
  1866. done
  1867. { echo "\\\\" ; echo "\\\\"; } >> "${commentfile}"
  1868. echo "|| Vote || Subsystem || Runtime || Comment ||" >> "${commentfile}"
  1869. i=0
  1870. until [[ $i -eq ${#JIRA_COMMENT_TABLE[@]} ]]; do
  1871. printf "%s\n" "${JIRA_COMMENT_TABLE[${i}]}" >> "${commentfile}"
  1872. ((i=i+1))
  1873. done
  1874. if [[ ${#JIRA_TEST_TABLE[@]} -gt 0 ]]; then
  1875. { echo "\\\\" ; echo "\\\\"; } >> "${commentfile}"
  1876. echo "|| Reason || Tests ||" >> "${commentfile}"
  1877. i=0
  1878. until [[ $i -eq ${#JIRA_TEST_TABLE[@]} ]]; do
  1879. printf "%s\n" "${JIRA_TEST_TABLE[${i}]}" >> "${commentfile}"
  1880. ((i=i+1))
  1881. done
  1882. fi
  1883. { echo "\\\\" ; echo "\\\\"; } >> "${commentfile}"
  1884. echo "|| Subsystem || Report/Notes ||" >> "${commentfile}"
  1885. i=0
  1886. until [[ $i -eq ${#JIRA_FOOTER_TABLE[@]} ]]; do
  1887. comment=$(echo "${JIRA_FOOTER_TABLE[${i}]}" |
  1888. ${SED} -e "s,@@BASE@@,${BUILD_URL}artifact/patchprocess,g")
  1889. printf "%s\n" "${comment}" >> "${commentfile}"
  1890. ((i=i+1))
  1891. done
  1892. printf "\n\nThis message was automatically generated.\n\n" >> "${commentfile}"
  1893. write_to_jira "${commentfile}"
  1894. }
  1895. ## @description Clean the filesystem as appropriate and then exit
  1896. ## @audience private
  1897. ## @stability evolving
  1898. ## @replaceable no
  1899. ## @param runresult
  1900. function cleanup_and_exit
  1901. {
  1902. local result=$1
  1903. if [[ ${JENKINS} == "true" ]] ; then
  1904. if [[ -e "${PATCH_DIR}" ]] ; then
  1905. hadoop_debug "mv ${PATCH_DIR} ${BASEDIR} "
  1906. if [[ -d "${PATCH_DIR}" ]]; then
  1907. mv "${PATCH_DIR}" "${BASEDIR}"
  1908. fi
  1909. fi
  1910. fi
  1911. big_console_header "Finished build."
  1912. # shellcheck disable=SC2086
  1913. exit ${result}
  1914. }
  1915. ## @description Driver to execute _postcheckout routines
  1916. ## @audience private
  1917. ## @stability evolving
  1918. ## @replaceable no
  1919. function postcheckout
  1920. {
  1921. local routine
  1922. local plugin
  1923. for routine in find_java_home verify_patch_file
  1924. do
  1925. verify_patchdir_still_exists
  1926. hadoop_debug "Running ${routine}"
  1927. ${routine}
  1928. (( RESULT = RESULT + $? ))
  1929. if [[ ${RESULT} != 0 ]] ; then
  1930. output_to_console 1
  1931. output_to_jira 1
  1932. cleanup_and_exit 1
  1933. fi
  1934. done
  1935. for plugin in ${PLUGINS}; do
  1936. verify_patchdir_still_exists
  1937. if declare -f ${plugin}_postcheckout >/dev/null 2>&1; then
  1938. hadoop_debug "Running ${plugin}_postcheckout"
  1939. #shellcheck disable=SC2086
  1940. ${plugin}_postcheckout
  1941. (( RESULT = RESULT + $? ))
  1942. if [[ ${RESULT} != 0 ]] ; then
  1943. output_to_console 1
  1944. output_to_jira 1
  1945. cleanup_and_exit 1
  1946. fi
  1947. fi
  1948. done
  1949. }
  1950. ## @description Driver to execute _preapply routines
  1951. ## @audience private
  1952. ## @stability evolving
  1953. ## @replaceable no
  1954. function preapply
  1955. {
  1956. local routine
  1957. local plugin
  1958. for routine in precheck_without_patch check_author \
  1959. check_modified_unittests
  1960. do
  1961. verify_patchdir_still_exists
  1962. hadoop_debug "Running ${routine}"
  1963. ${routine}
  1964. (( RESULT = RESULT + $? ))
  1965. done
  1966. for plugin in ${PLUGINS}; do
  1967. verify_patchdir_still_exists
  1968. if declare -f ${plugin}_preapply >/dev/null 2>&1; then
  1969. hadoop_debug "Running ${plugin}_preapply"
  1970. #shellcheck disable=SC2086
  1971. ${plugin}_preapply
  1972. (( RESULT = RESULT + $? ))
  1973. fi
  1974. done
  1975. }
  1976. ## @description Driver to execute _postapply routines
  1977. ## @audience private
  1978. ## @stability evolving
  1979. ## @replaceable no
  1980. function postapply
  1981. {
  1982. local routine
  1983. local plugin
  1984. local retval
  1985. compute_gitdiff "${GITDIFFLINES}"
  1986. check_javac
  1987. retval=$?
  1988. if [[ ${retval} -gt 1 ]] ; then
  1989. output_to_console 1
  1990. output_to_jira 1
  1991. cleanup_and_exit 1
  1992. fi
  1993. ((RESULT = RESULT + retval))
  1994. for routine in check_javadoc check_apachelicense check_site
  1995. do
  1996. verify_patchdir_still_exists
  1997. hadoop_debug "Running ${routine}"
  1998. $routine
  1999. (( RESULT = RESULT + $? ))
  2000. done
  2001. for plugin in ${PLUGINS}; do
  2002. verify_patchdir_still_exists
  2003. if declare -f ${plugin}_postapply >/dev/null 2>&1; then
  2004. hadoop_debug "Running ${plugin}_postapply"
  2005. #shellcheck disable=SC2086
  2006. ${plugin}_postapply
  2007. (( RESULT = RESULT + $? ))
  2008. fi
  2009. done
  2010. }
  2011. ## @description Driver to execute _postinstall routines
  2012. ## @audience private
  2013. ## @stability evolving
  2014. ## @replaceable no
  2015. function postinstall
  2016. {
  2017. local routine
  2018. local plugin
  2019. for routine in check_mvn_eclipse check_findbugs
  2020. do
  2021. verify_patchdir_still_exists
  2022. hadoop_debug "Running ${routine}"
  2023. ${routine}
  2024. (( RESULT = RESULT + $? ))
  2025. done
  2026. for plugin in ${PLUGINS}; do
  2027. verify_patchdir_still_exists
  2028. if declare -f ${plugin}_postinstall >/dev/null 2>&1; then
  2029. hadoop_debug "Running ${plugin}_postinstall"
  2030. #shellcheck disable=SC2086
  2031. ${plugin}_postinstall
  2032. (( RESULT = RESULT + $? ))
  2033. fi
  2034. done
  2035. }
  2036. ## @description Driver to execute _tests routines
  2037. ## @audience private
  2038. ## @stability evolving
  2039. ## @replaceable no
  2040. function runtests
  2041. {
  2042. local plugin
  2043. ### Run tests for Jenkins or if explictly asked for by a developer
  2044. if [[ ${JENKINS} == "true" || ${RUN_TESTS} == "true" ]] ; then
  2045. verify_patchdir_still_exists
  2046. check_unittests
  2047. (( RESULT = RESULT + $? ))
  2048. fi
  2049. for plugin in ${PLUGINS}; do
  2050. verify_patchdir_still_exists
  2051. if declare -f ${plugin}_tests >/dev/null 2>&1; then
  2052. hadoop_debug "Running ${plugin}_tests"
  2053. #shellcheck disable=SC2086
  2054. ${plugin}_tests
  2055. (( RESULT = RESULT + $? ))
  2056. fi
  2057. done
  2058. }
  2059. ## @description Import content from test-patch.d
  2060. ## @audience private
  2061. ## @stability evolving
  2062. ## @replaceable no
  2063. function importplugins
  2064. {
  2065. local i
  2066. local files
  2067. if [[ -d "${BINDIR}/test-patch.d" ]]; then
  2068. files=(${BINDIR}/test-patch.d/*.sh)
  2069. fi
  2070. for i in "${files[@]}"; do
  2071. hadoop_debug "Importing ${i}"
  2072. . "${i}"
  2073. done
  2074. }
  2075. ## @description Register test-patch.d plugins
  2076. ## @audience public
  2077. ## @stability stable
  2078. ## @replaceable no
  2079. function add_plugin
  2080. {
  2081. PLUGINS="${PLUGINS} $1"
  2082. }
  2083. ###############################################################################
  2084. ###############################################################################
  2085. ###############################################################################
  2086. big_console_header "Bootstrapping test harness"
  2087. setup_defaults
  2088. parse_args "$@"
  2089. importplugins
  2090. locate_patch
  2091. find_changed_files
  2092. determine_needed_tests
  2093. git_checkout
  2094. RESULT=$?
  2095. if [[ ${JENKINS} == "true" ]] ; then
  2096. if [[ ${RESULT} != 0 ]] ; then
  2097. exit 100
  2098. fi
  2099. fi
  2100. check_reexec
  2101. postcheckout
  2102. find_changed_modules
  2103. preapply
  2104. apply_patch_file
  2105. postapply
  2106. check_mvn_install
  2107. postinstall
  2108. runtests
  2109. close_jira_footer
  2110. close_jira_table
  2111. output_to_console ${RESULT}
  2112. output_to_jira ${RESULT}
  2113. cleanup_and_exit ${RESULT}