test-patch.sh 79 KB

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