hodRing.py 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930
  1. #Licensed to the Apache Software Foundation (ASF) under one
  2. #or more contributor license agreements. See the NOTICE file
  3. #distributed with this work for additional information
  4. #regarding copyright ownership. The ASF licenses this file
  5. #to you under the Apache License, Version 2.0 (the
  6. #"License"); you may not use this file except in compliance
  7. #with the License. You may obtain a copy of the License at
  8. # http://www.apache.org/licenses/LICENSE-2.0
  9. #Unless required by applicable law or agreed to in writing, software
  10. #distributed under the License is distributed on an "AS IS" BASIS,
  11. #WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. #See the License for the specific language governing permissions and
  13. #limitations under the License.
  14. #!/usr/bin/env python
  15. """hodring launches hadoop commands on work node and
  16. cleans up all the work dirs afterward
  17. """
  18. # -*- python -*-
  19. import os, sys, time, shutil, getpass, xml.dom.minidom, xml.dom.pulldom
  20. import socket, sets, urllib, csv, signal, pprint, random, re, httplib
  21. from xml.dom import getDOMImplementation
  22. from pprint import pformat
  23. from optparse import OptionParser
  24. from urlparse import urlparse
  25. from hodlib.Common.util import local_fqdn, parseEquals, getMapredSystemDirectory, isProcessRunning
  26. from hodlib.Common.tcp import tcpSocket, tcpError
  27. binfile = sys.path[0]
  28. libdir = os.path.dirname(binfile)
  29. sys.path.append(libdir)
  30. import hodlib.Common.logger
  31. from hodlib.GridServices.service import *
  32. from hodlib.Common.util import *
  33. from hodlib.Common.socketServers import threadedHTTPServer
  34. from hodlib.Common.hodsvc import hodBaseService
  35. from hodlib.Common.threads import simpleCommand
  36. from hodlib.Common.xmlrpc import hodXRClient
  37. mswindows = (sys.platform == "win32")
  38. originalcwd = os.getcwd()
  39. reHdfsURI = re.compile("hdfs://(.*?:\d+)(.*)")
  40. class CommandDesc:
  41. """A class that represents the commands that
  42. are run by hodring"""
  43. def __init__(self, dict, log):
  44. self.log = log
  45. self.log.debug("In command desc")
  46. self.log.debug("Done in command desc")
  47. dict.setdefault('argv', [])
  48. dict.setdefault('version', None)
  49. dict.setdefault('envs', {})
  50. dict.setdefault('workdirs', [])
  51. dict.setdefault('attrs', {})
  52. dict.setdefault('final-attrs', {})
  53. dict.setdefault('fg', False)
  54. dict.setdefault('ignorefailures', False)
  55. dict.setdefault('stdin', None)
  56. self.log.debug("Printing dict")
  57. self._checkRequired(dict)
  58. self.dict = dict
  59. def _checkRequired(self, dict):
  60. if 'name' not in dict:
  61. raise ValueError, "Command description lacks 'name'"
  62. if 'program' not in dict:
  63. raise ValueError, "Command description lacks 'program'"
  64. if 'pkgdirs' not in dict:
  65. raise ValueError, "Command description lacks 'pkgdirs'"
  66. def getName(self):
  67. return self.dict['name']
  68. def getProgram(self):
  69. return self.dict['program']
  70. def getArgv(self):
  71. return self.dict['argv']
  72. def getVersion(self):
  73. return self.dict['version']
  74. def getEnvs(self):
  75. return self.dict['envs']
  76. def getPkgDirs(self):
  77. return self.dict['pkgdirs']
  78. def getWorkDirs(self):
  79. return self.dict['workdirs']
  80. def getAttrs(self):
  81. return self.dict['attrs']
  82. def getfinalAttrs(self):
  83. return self.dict['final-attrs']
  84. def isForeground(self):
  85. return self.dict['fg']
  86. def isIgnoreFailures(self):
  87. return self.dict['ignorefailures']
  88. def getStdin(self):
  89. return self.dict['stdin']
  90. def parseDesc(str):
  91. dict = CommandDesc._parseMap(str)
  92. dict['argv'] = CommandDesc._parseList(dict['argv'])
  93. dict['envs'] = CommandDesc._parseMap(dict['envs'])
  94. dict['pkgdirs'] = CommandDesc._parseList(dict['pkgdirs'], ':')
  95. dict['workdirs'] = CommandDesc._parseList(dict['workdirs'], ':')
  96. dict['attrs'] = CommandDesc._parseMap(dict['attrs'])
  97. dict['final-attrs'] = CommandDesc._parseMap(dict['final-attrs'])
  98. return CommandDesc(dict)
  99. parseDesc = staticmethod(parseDesc)
  100. def _parseList(str, delim = ','):
  101. list = []
  102. for row in csv.reader([str], delimiter=delim, escapechar='\\',
  103. quoting=csv.QUOTE_NONE, doublequote=False):
  104. list.extend(row)
  105. return list
  106. _parseList = staticmethod(_parseList)
  107. def _parseMap(str):
  108. """Parses key value pairs"""
  109. dict = {}
  110. for row in csv.reader([str], escapechar='\\', quoting=csv.QUOTE_NONE, doublequote=False):
  111. for f in row:
  112. [k, v] = f.split('=', 1)
  113. dict[k] = v
  114. return dict
  115. _parseMap = staticmethod(_parseMap)
  116. class MRSystemDirectoryManager:
  117. """Class that is responsible for managing the MapReduce system directory"""
  118. def __init__(self, jtPid, mrSysDir, fsName, hadoopPath, log, retries=120):
  119. self.__jtPid = jtPid
  120. self.__mrSysDir = mrSysDir
  121. self.__fsName = fsName
  122. self.__hadoopPath = hadoopPath
  123. self.__log = log
  124. self.__retries = retries
  125. def toCleanupArgs(self):
  126. return " --jt-pid %s --mr-sys-dir %s --fs-name %s --hadoop-path %s " \
  127. % (self.__jtPid, self.__mrSysDir, self.__fsName, self.__hadoopPath)
  128. def removeMRSystemDirectory(self):
  129. jtActive = isProcessRunning(self.__jtPid)
  130. count = 0 # try for a max of a minute for the process to end
  131. while jtActive and (count<self.__retries):
  132. time.sleep(0.5)
  133. jtActive = isProcessRunning(self.__jtPid)
  134. count += 1
  135. if count == self.__retries:
  136. self.__log.warn('Job Tracker did not exit even after a minute. Not going to try and cleanup the system directory')
  137. return
  138. self.__log.debug('jt is now inactive')
  139. cmd = "%s dfs -fs hdfs://%s -rmr %s" % (self.__hadoopPath, self.__fsName, \
  140. self.__mrSysDir)
  141. self.__log.debug('Command to run to remove system directory: %s' % (cmd))
  142. try:
  143. hadoopCommand = simpleCommand('mr-sys-dir-cleaner', cmd)
  144. hadoopCommand.start()
  145. hadoopCommand.wait()
  146. hadoopCommand.join()
  147. ret = hadoopCommand.exit_code()
  148. if ret != 0:
  149. self.__log.warn("Error in removing MapReduce system directory '%s' from '%s' using path '%s'" \
  150. % (self.__mrSysDir, self.__fsName, self.__hadoopPath))
  151. self.__log.warn(pprint.pformat(hadoopCommand.output()))
  152. else:
  153. self.__log.info("Removed MapReduce system directory successfully.")
  154. except:
  155. self.__log.error('Exception while cleaning up MapReduce system directory. May not be cleaned up. %s', \
  156. get_exception_error_string())
  157. self.__log.debug(get_exception_string())
  158. def createMRSystemDirectoryManager(dict, log):
  159. keys = [ 'jt-pid', 'mr-sys-dir', 'fs-name', 'hadoop-path' ]
  160. for key in keys:
  161. if (not dict.has_key(key)) or (dict[key] is None):
  162. return None
  163. mrSysDirManager = MRSystemDirectoryManager(int(dict['jt-pid']), dict['mr-sys-dir'], \
  164. dict['fs-name'], dict['hadoop-path'], log)
  165. return mrSysDirManager
  166. class HadoopCommand:
  167. """Runs a single hadoop command"""
  168. def __init__(self, id, desc, tempdir, tardir, hadoopportrange, log, javahome,
  169. mrSysDir, restart=False):
  170. self.desc = desc
  171. self.log = log
  172. self.javahome = javahome
  173. self.__mrSysDir = mrSysDir
  174. self.program = desc.getProgram()
  175. self.name = desc.getName()
  176. self.workdirs = desc.getWorkDirs()
  177. self.hadoopdir = tempdir
  178. self.confdir = os.path.join(self.hadoopdir, '%d-%s' % (id, self.name),
  179. "confdir")
  180. self.logdir = os.path.join(self.hadoopdir, '%d-%s' % (id, self.name),
  181. "logdir")
  182. self.out = os.path.join(self.logdir, '%s.out' % self.name)
  183. self.err = os.path.join(self.logdir, '%s.err' % self.name)
  184. self.child = None
  185. self.restart = restart
  186. self.filledInKeyVals = []
  187. self.__hadoopPortRange = hadoopportrange
  188. self._createWorkDirs()
  189. self._createHadoopSiteXml()
  190. self._createHadoopLogDir()
  191. self.__hadoopThread = None
  192. self.stdErrContents = "" # store list of contents for returning to user
  193. def _createWorkDirs(self):
  194. for dir in self.workdirs:
  195. if os.path.exists(dir):
  196. if not os.access(dir, os.F_OK | os.R_OK | os.W_OK | os.X_OK):
  197. raise ValueError, "Workdir %s does not allow rwx permission." % (dir)
  198. continue
  199. try:
  200. os.makedirs(dir)
  201. except:
  202. pass
  203. def getFilledInKeyValues(self):
  204. return self.filledInKeyVals
  205. def createXML(self, doc, attr, topElement, final):
  206. for k,v in attr.iteritems():
  207. self.log.debug('_createHadoopSiteXml: ' + str(k) + " " + str(v))
  208. lowport, highport = self.__hadoopPortRange
  209. if ( v == "fillinport" ):
  210. v = "%d" % (ServiceUtil.getUniqRandomPort(low=lowport, high=highport, log=self.log))
  211. keyvalpair = ''
  212. if isinstance(v, (tuple, list)):
  213. for item in v:
  214. keyvalpair = "%s%s=%s," % (keyvalpair, k, item)
  215. keyvalpair = keyvalpair[:-1]
  216. else:
  217. keyvalpair = k + '=' + v
  218. self.filledInKeyVals.append(keyvalpair)
  219. if(k == "mapred.job.tracker"): # total hack for time's sake
  220. keyvalpair = k + "=" + v
  221. self.filledInKeyVals.append(keyvalpair)
  222. if ( v == "fillinhostport"):
  223. port = "%d" % (ServiceUtil.getUniqRandomPort(low=lowport, high=highport, log=self.log))
  224. self.log.debug('Setting hostname to: %s' % local_fqdn())
  225. v = local_fqdn() + ':' + port
  226. keyvalpair = ''
  227. if isinstance(v, (tuple, list)):
  228. for item in v:
  229. keyvalpair = "%s%s=%s," % (keyvalpair, k, item)
  230. keyvalpair = keyvalpair[:-1]
  231. else:
  232. keyvalpair = k + '=' + v
  233. self.filledInKeyVals.append(keyvalpair)
  234. if ( v == "fillindir"):
  235. v = self.__mrSysDir
  236. pass
  237. prop = None
  238. if isinstance(v, (tuple, list)):
  239. for item in v:
  240. prop = self._createXmlElement(doc, k, item, "No description", final)
  241. topElement.appendChild(prop)
  242. else:
  243. if k == 'fs.default.name':
  244. prop = self._createXmlElement(doc, k, "hdfs://" + v, "No description", final)
  245. else:
  246. prop = self._createXmlElement(doc, k, v, "No description", final)
  247. topElement.appendChild(prop)
  248. def _createHadoopSiteXml(self):
  249. if self.restart:
  250. if not os.path.exists(self.confdir):
  251. os.makedirs(self.confdir)
  252. else:
  253. assert os.path.exists(self.confdir) == False
  254. os.makedirs(self.confdir)
  255. implementation = getDOMImplementation()
  256. doc = implementation.createDocument('', 'configuration', None)
  257. comment = doc.createComment("This is an auto generated hadoop-site.xml, do not modify")
  258. topElement = doc.documentElement
  259. topElement.appendChild(comment)
  260. finalAttr = self.desc.getfinalAttrs()
  261. self.createXML(doc, finalAttr, topElement, True)
  262. attr = {}
  263. attr1 = self.desc.getAttrs()
  264. for k,v in attr1.iteritems():
  265. if not finalAttr.has_key(k):
  266. attr[k] = v
  267. self.createXML(doc, attr, topElement, False)
  268. siteName = os.path.join(self.confdir, "hadoop-site.xml")
  269. sitefile = file(siteName, 'w')
  270. print >> sitefile, topElement.toxml()
  271. sitefile.close()
  272. self.log.debug('created %s' % (siteName))
  273. def _createHadoopLogDir(self):
  274. if self.restart:
  275. if not os.path.exists(self.logdir):
  276. os.makedirs(self.logdir)
  277. else:
  278. assert os.path.exists(self.logdir) == False
  279. os.makedirs(self.logdir)
  280. def _createXmlElement(self, doc, name, value, description, final):
  281. prop = doc.createElement("property")
  282. nameP = doc.createElement("name")
  283. string = doc.createTextNode(name)
  284. nameP.appendChild(string)
  285. valueP = doc.createElement("value")
  286. string = doc.createTextNode(value)
  287. valueP.appendChild(string)
  288. desc = doc.createElement("description")
  289. string = doc.createTextNode(description)
  290. desc.appendChild(string)
  291. prop.appendChild(nameP)
  292. prop.appendChild(valueP)
  293. prop.appendChild(desc)
  294. if (final):
  295. felement = doc.createElement("final")
  296. string = doc.createTextNode("true")
  297. felement.appendChild(string)
  298. prop.appendChild(felement)
  299. pass
  300. return prop
  301. def getMRSystemDirectoryManager(self):
  302. return MRSystemDirectoryManager(self.__hadoopThread.getPid(), self.__mrSysDir, \
  303. self.desc.getfinalAttrs()['fs.default.name'], \
  304. self.path, self.log)
  305. def run(self, dir):
  306. status = True
  307. args = []
  308. desc = self.desc
  309. self.log.debug(pprint.pformat(desc.dict))
  310. self.log.debug("Got package dir of %s" % dir)
  311. self.path = os.path.join(dir, self.program)
  312. self.log.debug("path: %s" % self.path)
  313. args.append(self.path)
  314. args.extend(desc.getArgv())
  315. envs = desc.getEnvs()
  316. fenvs = os.environ
  317. for k, v in envs.iteritems():
  318. fenvs[k] = v
  319. if envs.has_key('HADOOP_OPTS'):
  320. fenvs['HADOOP_OPTS'] = envs['HADOOP_OPTS']
  321. self.log.debug("HADOOP_OPTS : %s" % fenvs['HADOOP_OPTS'])
  322. fenvs['JAVA_HOME'] = self.javahome
  323. fenvs['HADOOP_CONF_DIR'] = self.confdir
  324. fenvs['HADOOP_LOG_DIR'] = self.logdir
  325. self.log.info(pprint.pformat(fenvs))
  326. hadoopCommand = ''
  327. for item in args:
  328. hadoopCommand = "%s%s " % (hadoopCommand, item)
  329. # Redirecting output and error to self.out and self.err
  330. hadoopCommand = hadoopCommand + ' 1>%s 2>%s ' % (self.out, self.err)
  331. self.log.debug('running command: %s' % (hadoopCommand))
  332. self.log.debug('hadoop env: %s' % fenvs)
  333. self.log.debug('Command stdout will be redirected to %s ' % self.out + \
  334. 'and command stderr to %s' % self.err)
  335. self.__hadoopThread = simpleCommand('hadoop', hadoopCommand, env=fenvs)
  336. self.__hadoopThread.start()
  337. while self.__hadoopThread.stdin == None:
  338. time.sleep(.2)
  339. self.log.debug("hadoopThread still == None ...")
  340. input = desc.getStdin()
  341. self.log.debug("hadoop input: %s" % input)
  342. if input:
  343. if self.__hadoopThread.is_running():
  344. print >>self.__hadoopThread.stdin, input
  345. else:
  346. self.log.error("hadoop command failed to start")
  347. self.__hadoopThread.stdin.close()
  348. self.log.debug("isForground: %s" % desc.isForeground())
  349. if desc.isForeground():
  350. self.log.debug("Waiting on hadoop to finish...")
  351. self.__hadoopThread.wait()
  352. self.log.debug("Joining hadoop thread...")
  353. self.__hadoopThread.join()
  354. if self.__hadoopThread.exit_code() != 0:
  355. status = False
  356. else:
  357. status = self.getCommandStatus()
  358. self.log.debug("hadoop run status: %s" % status)
  359. if status == False:
  360. self.handleFailedCommand()
  361. if (status == True) or (not desc.isIgnoreFailures()):
  362. return status
  363. else:
  364. self.log.error("Ignoring Failure")
  365. return True
  366. def kill(self):
  367. self.__hadoopThread.kill()
  368. if self.__hadoopThread:
  369. self.__hadoopThread.join()
  370. def addCleanup(self, list):
  371. list.extend(self.workdirs)
  372. list.append(self.confdir)
  373. def getCommandStatus(self):
  374. status = True
  375. ec = self.__hadoopThread.exit_code()
  376. if (ec != 0) and (ec != None):
  377. status = False
  378. return status
  379. def handleFailedCommand(self):
  380. self.log.error('hadoop error: %s' % (
  381. self.__hadoopThread.exit_status_string()))
  382. # read the contents of redirected stderr to print information back to user
  383. if os.path.exists(self.err):
  384. f = None
  385. try:
  386. f = open(self.err)
  387. lines = f.readlines()
  388. # format
  389. for line in lines:
  390. self.stdErrContents = "%s%s" % (self.stdErrContents, line)
  391. finally:
  392. if f is not None:
  393. f.close()
  394. self.log.error('See %s.out and/or %s.err for details. They are ' % \
  395. (self.name, self.name) + \
  396. 'located at subdirectories under either ' + \
  397. 'hodring.work-dirs or hodring.log-destination-uri.')
  398. class HodRing(hodBaseService):
  399. """The main class for hodring that
  400. polls the commands it runs"""
  401. def __init__(self, config):
  402. hodBaseService.__init__(self, 'hodring', config['hodring'])
  403. self.log = self.logs['main']
  404. self._http = None
  405. self.__pkg = None
  406. self.__pkgDir = None
  407. self.__tempDir = None
  408. self.__running = {}
  409. self.__hadoopLogDirs = []
  410. self.__init_temp_dir()
  411. def __init_temp_dir(self):
  412. self.__tempDir = os.path.join(self._cfg['temp-dir'],
  413. "%s.%s.hodring" % (self._cfg['userid'],
  414. self._cfg['service-id']))
  415. if not os.path.exists(self.__tempDir):
  416. os.makedirs(self.__tempDir)
  417. os.chdir(self.__tempDir)
  418. def __fetch(self, url, spath):
  419. retry = 3
  420. success = False
  421. while (retry != 0 and success != True):
  422. try:
  423. input = urllib.urlopen(url)
  424. bufsz = 81920
  425. buf = input.read(bufsz)
  426. out = open(spath, 'w')
  427. while len(buf) > 0:
  428. out.write(buf)
  429. buf = input.read(bufsz)
  430. input.close()
  431. out.close()
  432. success = True
  433. except:
  434. self.log.debug("Failed to copy file")
  435. retry = retry - 1
  436. if (retry == 0 and success != True):
  437. raise IOError, "Failed to copy the files"
  438. def __get_name(self, addr):
  439. parsedUrl = urlparse(addr)
  440. path = parsedUrl[2]
  441. split = path.split('/', 1)
  442. return split[1]
  443. def __get_dir(self, name):
  444. """Return the root directory inside the tarball
  445. specified by name. Assumes that the tarball begins
  446. with a root directory."""
  447. import tarfile
  448. myTarFile = tarfile.open(name)
  449. hadoopPackage = myTarFile.getnames()[0]
  450. self.log.debug("tarball name : %s hadoop package name : %s" %(name,hadoopPackage))
  451. return hadoopPackage
  452. def getRunningValues(self):
  453. return self.__running.values()
  454. def getTempDir(self):
  455. return self.__tempDir
  456. def getHadoopLogDirs(self):
  457. return self.__hadoopLogDirs
  458. def __download_package(self, ringClient):
  459. self.log.debug("Found download address: %s" %
  460. self._cfg['download-addr'])
  461. try:
  462. addr = 'none'
  463. downloadTime = self._cfg['tarball-retry-initial-time'] # download time depends on tarball size and network bandwidth
  464. increment = 0
  465. addr = ringClient.getTarList(self.hostname)
  466. while(addr == 'none'):
  467. rand = self._cfg['tarball-retry-initial-time'] + increment + \
  468. random.uniform(0,self._cfg['tarball-retry-interval'])
  469. increment = increment + 1
  470. self.log.debug("got no tarball. Retrying again in %s seconds." % rand)
  471. time.sleep(rand)
  472. addr = ringClient.getTarList(self.hostname)
  473. self.log.debug("got this address %s" % addr)
  474. tarName = self.__get_name(addr)
  475. self.log.debug("tar package name: %s" % tarName)
  476. fetchPath = os.path.join(os.getcwd(), tarName)
  477. self.log.debug("fetch path: %s" % fetchPath)
  478. self.__fetch(addr, fetchPath)
  479. self.log.debug("done fetching")
  480. tarUrl = "http://%s:%d/%s" % (self._http.server_address[0],
  481. self._http.server_address[1],
  482. tarName)
  483. try:
  484. ringClient.registerTarSource(self.hostname, tarUrl,addr)
  485. #ringClient.tarDone(addr)
  486. except KeyError, e:
  487. self.log.error("registerTarSource and tarDone failed: ", e)
  488. raise KeyError(e)
  489. check = untar(fetchPath, os.getcwd())
  490. if (check == False):
  491. raise IOError, "Untarring failed."
  492. self.__pkg = self.__get_dir(tarName)
  493. self.__pkgDir = os.path.join(os.getcwd(), self.__pkg)
  494. except Exception, e:
  495. self.log.error("Failed download tar package: %s" %
  496. get_exception_error_string())
  497. raise Exception(e)
  498. def __run_hadoop_commands(self, restart=True):
  499. id = 0
  500. for desc in self._cfg['commanddesc']:
  501. self.log.debug(pprint.pformat(desc.dict))
  502. mrSysDir = getMapredSystemDirectory(self._cfg['mapred-system-dir-root'],
  503. self._cfg['userid'], self._cfg['service-id'])
  504. self.log.debug('mrsysdir is %s' % mrSysDir)
  505. cmd = HadoopCommand(id, desc, self.__tempDir, self.__pkgDir, self._cfg['hadoop-port-range'], self.log,
  506. self._cfg['java-home'], mrSysDir, restart)
  507. self.__hadoopLogDirs.append(cmd.logdir)
  508. self.log.debug("hadoop log directory: %s" % self.__hadoopLogDirs)
  509. try:
  510. # if the tarball isn't there, we use the pkgs dir given.
  511. if self.__pkgDir == None:
  512. pkgdir = desc.getPkgDirs()
  513. else:
  514. pkgdir = self.__pkgDir
  515. self.log.debug('This is the packcage dir %s ' % (pkgdir))
  516. if not cmd.run(pkgdir):
  517. addnInfo = ""
  518. if cmd.stdErrContents is not "":
  519. addnInfo = " Information from stderr of the command:\n%s" % (cmd.stdErrContents)
  520. raise Exception("Could not launch the %s using %s/bin/hadoop.%s" % (desc.getName(), pkgdir, addnInfo))
  521. except Exception, e:
  522. self.log.debug("Exception running hadoop command: %s\n%s" % (get_exception_error_string(), get_exception_string()))
  523. self.__running[id] = cmd
  524. raise Exception(e)
  525. id += 1
  526. if desc.isForeground():
  527. continue
  528. self.__running[id-1] = cmd
  529. # ok.. now command is running. If this HodRing got jobtracker,
  530. # Check if it is ready for accepting jobs, and then only return
  531. self.__check_jobtracker(desc, id-1, pkgdir)
  532. def __check_jobtracker(self, desc, id, pkgdir):
  533. # Check jobtracker status. Return properly if it is ready to accept jobs.
  534. # Currently Checks for Jetty to come up, the last thing that can be checked
  535. # before JT completes initialisation. To be perfectly reliable, we need
  536. # hadoop support
  537. name = desc.getName()
  538. if name == 'jobtracker':
  539. # Yes I am the Jobtracker
  540. self.log.debug("Waiting for jobtracker to initialise")
  541. version = desc.getVersion()
  542. self.log.debug("jobtracker version : %s" % version)
  543. hadoopCmd = self.getRunningValues()[id]
  544. attrs = hadoopCmd.getFilledInKeyValues()
  545. attrs = parseEquals(attrs)
  546. jobTrackerAddr = attrs['mapred.job.tracker']
  547. self.log.debug("jobtracker rpc server : %s" % jobTrackerAddr)
  548. if version < 16:
  549. jettyAddr = jobTrackerAddr.split(':')[0] + ':' + \
  550. attrs['mapred.job.tracker.info.port']
  551. else:
  552. jettyAddr = attrs['mapred.job.tracker.http.address']
  553. self.log.debug("Jobtracker jetty : %s" % jettyAddr)
  554. # Check for Jetty to come up
  555. # For this do a http head, and then look at the status
  556. defaultTimeout = socket.getdefaulttimeout()
  557. # socket timeout isn`t exposed at httplib level. Setting explicitly.
  558. socket.setdefaulttimeout(1)
  559. sleepTime = 0.5
  560. jettyStatus = False
  561. jettyStatusmsg = ""
  562. while sleepTime <= 32:
  563. # There is a possibility that the command might fail after a while.
  564. # This code will check if the command failed so that a better
  565. # error message can be returned to the user.
  566. if not hadoopCmd.getCommandStatus():
  567. self.log.critical('Hadoop command found to have failed when ' \
  568. 'checking for jobtracker status')
  569. hadoopCmd.handleFailedCommand()
  570. addnInfo = ""
  571. if hadoopCmd.stdErrContents is not "":
  572. addnInfo = " Information from stderr of the command:\n%s" \
  573. % (hadoopCmd.stdErrContents)
  574. raise Exception("Could not launch the %s using %s/bin/hadoop.%s" \
  575. % (desc.getName(), pkgdir, addnInfo))
  576. try:
  577. jettyConn = httplib.HTTPConnection(jettyAddr)
  578. jettyConn.request("HEAD", "/jobtracker.jsp")
  579. # httplib inherently retries the following till socket timeout
  580. resp = jettyConn.getresponse()
  581. if resp.status != 200:
  582. # Some problem?
  583. jettyStatus = False
  584. jettyStatusmsg = "Jetty gave a non-200 response to a HTTP-HEAD" +\
  585. " request. HTTP Status (Code, Msg): (%s, %s)" % \
  586. ( resp.status, resp.reason )
  587. break
  588. else:
  589. self.log.info("Jetty returned a 200 status (%s)" % resp.reason)
  590. self.log.info("JobTracker successfully initialised")
  591. return
  592. except socket.error:
  593. self.log.debug("Jetty gave a socket error. Sleeping for %s" \
  594. % sleepTime)
  595. time.sleep(sleepTime)
  596. sleepTime = sleepTime * 2
  597. except Exception, e:
  598. jettyStatus = False
  599. jettyStatusmsg = ("Process(possibly other than jetty) running on" + \
  600. " port assigned to jetty is returning invalid http response")
  601. break
  602. socket.setdefaulttimeout(defaultTimeout)
  603. if not jettyStatus:
  604. self.log.critical("Jobtracker failed to initialise.")
  605. if jettyStatusmsg:
  606. self.log.critical( "Reason: %s" % jettyStatusmsg )
  607. else: self.log.critical( "Reason: Jetty failed to give response")
  608. raise Exception("JobTracker failed to initialise")
  609. def stop(self):
  610. self.log.debug("Entered hodring stop.")
  611. if self._http:
  612. self.log.debug("stopping http server...")
  613. self._http.stop()
  614. self.log.debug("call hodsvcrgy stop...")
  615. hodBaseService.stop(self)
  616. def _xr_method_clusterStart(self, initialize=True):
  617. return self.clusterStart(initialize)
  618. def _xr_method_clusterStop(self):
  619. return self.clusterStop()
  620. def start(self):
  621. """Run and maintain hodring commands"""
  622. try:
  623. if self._cfg.has_key('download-addr'):
  624. self._http = threadedHTTPServer('', self._cfg['http-port-range'])
  625. self.log.info("Starting http server...")
  626. self._http.serve_forever()
  627. self.log.debug("http://%s:%d" % (self._http.server_address[0],
  628. self._http.server_address[1]))
  629. hodBaseService.start(self)
  630. ringXRAddress = None
  631. if self._cfg.has_key('ringmaster-xrs-addr'):
  632. ringXRAddress = "http://%s:%s/" % (self._cfg['ringmaster-xrs-addr'][0],
  633. self._cfg['ringmaster-xrs-addr'][1])
  634. self.log.debug("Ringmaster at %s" % ringXRAddress)
  635. self.log.debug("Creating service registry XML-RPC client.")
  636. serviceClient = hodXRClient(to_http_url(
  637. self._cfg['svcrgy-addr']))
  638. if ringXRAddress == None:
  639. self.log.info("Did not get ringmaster XML-RPC address. Fetching information from service registry.")
  640. ringList = serviceClient.getServiceInfo(self._cfg['userid'],
  641. self._cfg['service-id'], 'ringmaster', 'hod')
  642. self.log.debug(pprint.pformat(ringList))
  643. if len(ringList):
  644. if isinstance(ringList, list):
  645. ringXRAddress = ringList[0]['xrs']
  646. count = 0
  647. while (ringXRAddress == None and count < 3000):
  648. ringList = serviceClient.getServiceInfo(self._cfg['userid'],
  649. self._cfg['service-id'], 'ringmaster', 'hod')
  650. if len(ringList):
  651. if isinstance(ringList, list):
  652. ringXRAddress = ringList[0]['xrs']
  653. count = count + 1
  654. time.sleep(.2)
  655. if ringXRAddress == None:
  656. raise Exception("Could not get ringmaster XML-RPC server address.")
  657. self.log.debug("Creating ringmaster XML-RPC client.")
  658. ringClient = hodXRClient(ringXRAddress)
  659. id = self.hostname + "_" + str(os.getpid())
  660. if 'download-addr' in self._cfg:
  661. self.__download_package(ringClient)
  662. else:
  663. self.log.debug("Did not find a download address.")
  664. cmdlist = []
  665. firstTime = True
  666. increment = 0
  667. hadoopStartupTime = 2
  668. cmdlist = ringClient.getCommand(id)
  669. while (cmdlist == []):
  670. if firstTime:
  671. sleepTime = increment + self._cfg['cmd-retry-initial-time'] + hadoopStartupTime\
  672. + random.uniform(0,self._cfg['cmd-retry-interval'])
  673. firstTime = False
  674. else:
  675. sleepTime = increment + self._cfg['cmd-retry-initial-time'] + \
  676. + random.uniform(0,self._cfg['cmd-retry-interval'])
  677. self.log.debug("Did not get command list. Waiting for %s seconds." % (sleepTime))
  678. time.sleep(sleepTime)
  679. increment = increment + 1
  680. cmdlist = ringClient.getCommand(id)
  681. self.log.debug(pformat(cmdlist))
  682. cmdDescs = []
  683. for cmds in cmdlist:
  684. cmdDescs.append(CommandDesc(cmds['dict'], self.log))
  685. self._cfg['commanddesc'] = cmdDescs
  686. self.log.info("Running hadoop commands...")
  687. self.__run_hadoop_commands(False)
  688. masterParams = []
  689. for k, cmd in self.__running.iteritems():
  690. masterParams.extend(cmd.filledInKeyVals)
  691. self.log.debug("printing getparams")
  692. self.log.debug(pformat(id))
  693. self.log.debug(pformat(masterParams))
  694. # when this is on a required host, the ringMaster already has our masterParams
  695. if(len(masterParams) > 0):
  696. ringClient.addMasterParams(id, masterParams)
  697. except Exception, e:
  698. raise Exception(e)
  699. def clusterStart(self, initialize=True):
  700. """Start a stopped mapreduce/dfs cluster"""
  701. if initialize:
  702. self.log.debug('clusterStart Method Invoked - Initialize')
  703. else:
  704. self.log.debug('clusterStart Method Invoked - No Initialize')
  705. try:
  706. self.log.debug("Creating service registry XML-RPC client.")
  707. serviceClient = hodXRClient(to_http_url(self._cfg['svcrgy-addr']),
  708. None, None, 0, 0, 0)
  709. self.log.info("Fetching ringmaster information from service registry.")
  710. count = 0
  711. ringXRAddress = None
  712. while (ringXRAddress == None and count < 3000):
  713. ringList = serviceClient.getServiceInfo(self._cfg['userid'],
  714. self._cfg['service-id'], 'ringmaster', 'hod')
  715. if len(ringList):
  716. if isinstance(ringList, list):
  717. ringXRAddress = ringList[0]['xrs']
  718. count = count + 1
  719. if ringXRAddress == None:
  720. raise Exception("Could not get ringmaster XML-RPC server address.")
  721. self.log.debug("Creating ringmaster XML-RPC client.")
  722. ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0)
  723. id = self.hostname + "_" + str(os.getpid())
  724. cmdlist = []
  725. if initialize:
  726. if 'download-addr' in self._cfg:
  727. self.__download_package(ringClient)
  728. else:
  729. self.log.debug("Did not find a download address.")
  730. while (cmdlist == []):
  731. cmdlist = ringClient.getCommand(id)
  732. else:
  733. while (cmdlist == []):
  734. cmdlist = ringClient.getAdminCommand(id)
  735. self.log.debug(pformat(cmdlist))
  736. cmdDescs = []
  737. for cmds in cmdlist:
  738. cmdDescs.append(CommandDesc(cmds['dict'], self.log))
  739. self._cfg['commanddesc'] = cmdDescs
  740. if initialize:
  741. self.log.info("Running hadoop commands again... - Initialize")
  742. self.__run_hadoop_commands()
  743. masterParams = []
  744. for k, cmd in self.__running.iteritems():
  745. self.log.debug(cmd)
  746. masterParams.extend(cmd.filledInKeyVals)
  747. self.log.debug("printing getparams")
  748. self.log.debug(pformat(id))
  749. self.log.debug(pformat(masterParams))
  750. # when this is on a required host, the ringMaster already has our masterParams
  751. if(len(masterParams) > 0):
  752. ringClient.addMasterParams(id, masterParams)
  753. else:
  754. self.log.info("Running hadoop commands again... - No Initialize")
  755. self.__run_hadoop_commands()
  756. except:
  757. self.log.error(get_exception_string())
  758. return True
  759. def clusterStop(self):
  760. """Stop a running mapreduce/dfs cluster without stopping the hodring"""
  761. self.log.debug('clusterStop Method Invoked')
  762. try:
  763. for cmd in self.__running.values():
  764. cmd.kill()
  765. self.__running = {}
  766. except:
  767. self.log.error(get_exception_string())
  768. return True