client.py 120 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510351135123513351435153516351735183519352035213522352335243525352635273528352935303531353235333534353535363537353835393540354135423543354435453546354735483549355035513552355335543555355635573558355935603561356235633564356535663567356835693570357135723573357435753576
  1. # client.py -- Implementation of the client side git protocols
  2. # Copyright (C) 2008-2013 Jelmer Vernooij <jelmer@jelmer.uk>
  3. #
  4. # SPDX-License-Identifier: Apache-2.0 OR GPL-2.0-or-later
  5. # Dulwich is dual-licensed under the Apache License, Version 2.0 and the GNU
  6. # General Public License as published by the Free Software Foundation; version 2.0
  7. # or (at your option) any later version. You can redistribute it and/or
  8. # modify it under the terms of either of these two licenses.
  9. #
  10. # Unless required by applicable law or agreed to in writing, software
  11. # distributed under the License is distributed on an "AS IS" BASIS,
  12. # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  13. # See the License for the specific language governing permissions and
  14. # limitations under the License.
  15. #
  16. # You should have received a copy of the licenses; if not, see
  17. # <http://www.gnu.org/licenses/> for a copy of the GNU General Public License
  18. # and <http://www.apache.org/licenses/LICENSE-2.0> for a copy of the Apache
  19. # License, Version 2.0.
  20. #
  21. """Client side support for the Git protocol.
  22. The Dulwich client supports the following capabilities:
  23. * thin-pack
  24. * multi_ack_detailed
  25. * multi_ack
  26. * side-band-64k
  27. * ofs-delta
  28. * quiet
  29. * report-status
  30. * delete-refs
  31. * shallow
  32. Known capabilities that are not supported:
  33. * no-progress
  34. * include-tag
  35. """
  36. import copy
  37. import functools
  38. import logging
  39. import os
  40. import select
  41. import socket
  42. import subprocess
  43. import sys
  44. from collections.abc import Iterable, Iterator
  45. from contextlib import closing
  46. from io import BufferedReader, BytesIO
  47. from typing import (
  48. IO,
  49. TYPE_CHECKING,
  50. Callable,
  51. ClassVar,
  52. Optional,
  53. Union,
  54. )
  55. from urllib.parse import quote as urlquote
  56. from urllib.parse import unquote as urlunquote
  57. from urllib.parse import urljoin, urlparse, urlunparse, urlunsplit
  58. if TYPE_CHECKING:
  59. import urllib3
  60. import dulwich
  61. from .config import Config, apply_instead_of, get_xdg_config_home_path
  62. from .errors import GitProtocolError, NotGitRepository, SendPackError
  63. from .pack import (
  64. PACK_SPOOL_FILE_MAX_SIZE,
  65. PackChunkGenerator,
  66. PackData,
  67. UnpackedObject,
  68. write_pack_from_container,
  69. )
  70. from .protocol import (
  71. _RBUFSIZE,
  72. CAPABILITIES_REF,
  73. CAPABILITY_AGENT,
  74. CAPABILITY_DELETE_REFS,
  75. CAPABILITY_FETCH,
  76. CAPABILITY_FILTER,
  77. CAPABILITY_INCLUDE_TAG,
  78. CAPABILITY_MULTI_ACK,
  79. CAPABILITY_MULTI_ACK_DETAILED,
  80. CAPABILITY_OFS_DELTA,
  81. CAPABILITY_QUIET,
  82. CAPABILITY_REPORT_STATUS,
  83. CAPABILITY_SHALLOW,
  84. CAPABILITY_SIDE_BAND_64K,
  85. CAPABILITY_SYMREF,
  86. CAPABILITY_THIN_PACK,
  87. COMMAND_DEEPEN,
  88. COMMAND_DONE,
  89. COMMAND_HAVE,
  90. COMMAND_SHALLOW,
  91. COMMAND_UNSHALLOW,
  92. COMMAND_WANT,
  93. DEFAULT_GIT_PROTOCOL_VERSION_FETCH,
  94. DEFAULT_GIT_PROTOCOL_VERSION_SEND,
  95. GIT_PROTOCOL_VERSIONS,
  96. KNOWN_RECEIVE_CAPABILITIES,
  97. KNOWN_UPLOAD_CAPABILITIES,
  98. SIDE_BAND_CHANNEL_DATA,
  99. SIDE_BAND_CHANNEL_FATAL,
  100. SIDE_BAND_CHANNEL_PROGRESS,
  101. TCP_GIT_PORT,
  102. ZERO_SHA,
  103. HangupException,
  104. PktLineParser,
  105. Protocol,
  106. agent_string,
  107. capability_agent,
  108. extract_capabilities,
  109. extract_capability_names,
  110. filter_ref_prefix,
  111. parse_capability,
  112. pkt_line,
  113. pkt_seq,
  114. )
  115. from .refs import (
  116. PEELED_TAG_SUFFIX,
  117. SYMREF,
  118. Ref,
  119. _import_remote_refs,
  120. _set_default_branch,
  121. _set_head,
  122. _set_origin_head,
  123. read_info_refs,
  124. split_peeled_refs,
  125. )
  126. from .repo import BaseRepo, Repo
  127. # Default ref prefix, used if none is specified.
  128. # GitHub defaults to just sending HEAD if no ref-prefix is
  129. # specified, so explicitly request all refs to match
  130. # behaviour with v1 when no ref-prefix is specified.
  131. DEFAULT_REF_PREFIX = [b"HEAD", b"refs/"]
  132. ObjectID = bytes
  133. logger = logging.getLogger(__name__)
  134. class InvalidWants(Exception):
  135. """Invalid wants."""
  136. def __init__(self, wants) -> None:
  137. """Initialize InvalidWants exception.
  138. Args:
  139. wants: List of invalid wants
  140. """
  141. Exception.__init__(
  142. self, f"requested wants not in server provided refs: {wants!r}"
  143. )
  144. class HTTPUnauthorized(Exception):
  145. """Raised when authentication fails."""
  146. def __init__(self, www_authenticate, url) -> None:
  147. """Initialize HTTPUnauthorized exception.
  148. Args:
  149. www_authenticate: WWW-Authenticate header value
  150. url: URL that requires authentication
  151. """
  152. Exception.__init__(self, "No valid credentials provided")
  153. self.www_authenticate = www_authenticate
  154. self.url = url
  155. class HTTPProxyUnauthorized(Exception):
  156. """Raised when proxy authentication fails."""
  157. def __init__(self, proxy_authenticate, url) -> None:
  158. """Initialize HTTPProxyUnauthorized exception.
  159. Args:
  160. proxy_authenticate: Proxy-Authenticate header value
  161. url: URL that requires proxy authentication
  162. """
  163. Exception.__init__(self, "No valid proxy credentials provided")
  164. self.proxy_authenticate = proxy_authenticate
  165. self.url = url
  166. def _fileno_can_read(fileno):
  167. """Check if a file descriptor is readable."""
  168. return len(select.select([fileno], [], [], 0)[0]) > 0
  169. def _win32_peek_avail(handle):
  170. """Wrapper around PeekNamedPipe to check how many bytes are available."""
  171. from ctypes import byref, windll, wintypes
  172. c_avail = wintypes.DWORD()
  173. c_message = wintypes.DWORD()
  174. success = windll.kernel32.PeekNamedPipe(
  175. handle, None, 0, None, byref(c_avail), byref(c_message)
  176. )
  177. if not success:
  178. raise OSError(wintypes.GetLastError())
  179. return c_avail.value
  180. COMMON_CAPABILITIES = [CAPABILITY_OFS_DELTA, CAPABILITY_SIDE_BAND_64K]
  181. UPLOAD_CAPABILITIES = [
  182. CAPABILITY_THIN_PACK,
  183. CAPABILITY_MULTI_ACK,
  184. CAPABILITY_MULTI_ACK_DETAILED,
  185. CAPABILITY_SHALLOW,
  186. *COMMON_CAPABILITIES,
  187. ]
  188. RECEIVE_CAPABILITIES = [
  189. CAPABILITY_REPORT_STATUS,
  190. CAPABILITY_DELETE_REFS,
  191. *COMMON_CAPABILITIES,
  192. ]
  193. class ReportStatusParser:
  194. """Handle status as reported by servers with 'report-status' capability."""
  195. def __init__(self) -> None:
  196. """Initialize ReportStatusParser."""
  197. self._done = False
  198. self._pack_status = None
  199. self._ref_statuses: list[bytes] = []
  200. def check(self):
  201. """Check if there were any errors and, if so, raise exceptions.
  202. Raises:
  203. SendPackError: Raised when the server could not unpack
  204. Returns:
  205. iterator over refs
  206. """
  207. if self._pack_status not in (b"unpack ok", None):
  208. raise SendPackError(self._pack_status)
  209. for status in self._ref_statuses:
  210. try:
  211. status, rest = status.split(b" ", 1)
  212. except ValueError:
  213. # malformed response, move on to the next one
  214. continue
  215. if status == b"ng":
  216. ref, error = rest.split(b" ", 1)
  217. yield ref, error.decode("utf-8")
  218. elif status == b"ok":
  219. yield rest, None
  220. else:
  221. raise GitProtocolError(f"invalid ref status {status!r}")
  222. def handle_packet(self, pkt) -> None:
  223. """Handle a packet.
  224. Raises:
  225. GitProtocolError: Raised when packets are received after a flush
  226. packet.
  227. """
  228. if self._done:
  229. raise GitProtocolError("received more data after status report")
  230. if pkt is None:
  231. self._done = True
  232. return
  233. if self._pack_status is None:
  234. self._pack_status = pkt.strip()
  235. else:
  236. ref_status = pkt.strip()
  237. self._ref_statuses.append(ref_status)
  238. def negotiate_protocol_version(proto) -> int:
  239. """Negotiate the protocol version to use.
  240. Args:
  241. proto: Protocol instance to negotiate with
  242. Returns: Protocol version (0, 1, or 2)
  243. """
  244. pkt = proto.read_pkt_line()
  245. if pkt is not None and pkt.strip() == b"version 2":
  246. return 2
  247. proto.unread_pkt_line(pkt)
  248. return 0
  249. def read_server_capabilities(pkt_seq):
  250. """Read server capabilities from a packet sequence.
  251. Args:
  252. pkt_seq: Sequence of packets from server
  253. Returns: Set of server capabilities
  254. """
  255. server_capabilities = []
  256. for pkt in pkt_seq:
  257. server_capabilities.append(pkt)
  258. return set(server_capabilities)
  259. def read_pkt_refs_v2(
  260. pkt_seq,
  261. ) -> tuple[dict[bytes, bytes], dict[bytes, bytes], dict[bytes, bytes]]:
  262. """Read packet references in protocol v2 format.
  263. Args:
  264. pkt_seq: Sequence of packets
  265. Returns: Tuple of (refs dict, symrefs dict, peeled dict)
  266. """
  267. refs = {}
  268. symrefs = {}
  269. peeled = {}
  270. # Receive refs from server
  271. for pkt in pkt_seq:
  272. parts = pkt.rstrip(b"\n").split(b" ")
  273. sha = parts[0]
  274. if sha == b"unborn":
  275. sha = None
  276. ref = parts[1]
  277. for part in parts[2:]:
  278. if part.startswith(b"peeled:"):
  279. peeled[ref] = part[7:]
  280. elif part.startswith(b"symref-target:"):
  281. symrefs[ref] = part[14:]
  282. else:
  283. logging.warning("unknown part in pkt-ref: %s", part)
  284. refs[ref] = sha
  285. return refs, symrefs, peeled
  286. def read_pkt_refs_v1(pkt_seq) -> tuple[dict[bytes, bytes], set[bytes]]:
  287. """Read packet references in protocol v1 format.
  288. Args:
  289. pkt_seq: Sequence of packets
  290. Returns: Tuple of (refs dict, server capabilities set)
  291. """
  292. server_capabilities = None
  293. refs = {}
  294. # Receive refs from server
  295. for pkt in pkt_seq:
  296. (sha, ref) = pkt.rstrip(b"\n").split(None, 1)
  297. if sha == b"ERR":
  298. raise GitProtocolError(ref.decode("utf-8", "replace"))
  299. if server_capabilities is None:
  300. (ref, server_capabilities) = extract_capabilities(ref)
  301. refs[ref] = sha
  302. if len(refs) == 0:
  303. return {}, set()
  304. if refs == {CAPABILITIES_REF: ZERO_SHA}:
  305. refs = {}
  306. assert server_capabilities is not None
  307. return refs, set(server_capabilities)
  308. class _DeprecatedDictProxy:
  309. """Base class for result objects that provide deprecated dict-like interface."""
  310. _FORWARDED_ATTRS: ClassVar[set[str]] = {
  311. "clear",
  312. "copy",
  313. "fromkeys",
  314. "get",
  315. "items",
  316. "keys",
  317. "pop",
  318. "popitem",
  319. "setdefault",
  320. "update",
  321. "values",
  322. "viewitems",
  323. "viewkeys",
  324. "viewvalues",
  325. }
  326. def _warn_deprecated(self) -> None:
  327. import warnings
  328. warnings.warn(
  329. f"Use {self.__class__.__name__}.refs instead.",
  330. DeprecationWarning,
  331. stacklevel=3,
  332. )
  333. def __contains__(self, name) -> bool:
  334. self._warn_deprecated()
  335. return name in self.refs
  336. def __getitem__(self, name):
  337. self._warn_deprecated()
  338. return self.refs[name]
  339. def __len__(self) -> int:
  340. self._warn_deprecated()
  341. return len(self.refs)
  342. def __iter__(self):
  343. self._warn_deprecated()
  344. return iter(self.refs)
  345. def __getattribute__(self, name):
  346. # Avoid infinite recursion by checking against class variable directly
  347. if name != "_FORWARDED_ATTRS" and name in type(self)._FORWARDED_ATTRS:
  348. self._warn_deprecated()
  349. # Direct attribute access to avoid recursion
  350. refs = object.__getattribute__(self, "refs")
  351. return getattr(refs, name)
  352. return super().__getattribute__(name)
  353. class FetchPackResult(_DeprecatedDictProxy):
  354. """Result of a fetch-pack operation.
  355. Attributes:
  356. refs: Dictionary with all remote refs
  357. symrefs: Dictionary with remote symrefs
  358. agent: User agent string
  359. """
  360. def __init__(
  361. self, refs, symrefs, agent, new_shallow=None, new_unshallow=None
  362. ) -> None:
  363. """Initialize FetchPackResult.
  364. Args:
  365. refs: Dictionary with all remote refs
  366. symrefs: Dictionary with remote symrefs
  367. agent: User agent string
  368. new_shallow: New shallow commits
  369. new_unshallow: New unshallow commits
  370. """
  371. self.refs = refs
  372. self.symrefs = symrefs
  373. self.agent = agent
  374. self.new_shallow = new_shallow
  375. self.new_unshallow = new_unshallow
  376. def __eq__(self, other):
  377. """Check equality with another FetchPackResult."""
  378. if isinstance(other, dict):
  379. self._warn_deprecated()
  380. return self.refs == other
  381. return (
  382. self.refs == other.refs
  383. and self.symrefs == other.symrefs
  384. and self.agent == other.agent
  385. )
  386. def __repr__(self) -> str:
  387. """Return string representation of FetchPackResult."""
  388. return f"{self.__class__.__name__}({self.refs!r}, {self.symrefs!r}, {self.agent!r})"
  389. class LsRemoteResult(_DeprecatedDictProxy):
  390. """Result of a ls-remote operation.
  391. Attributes:
  392. refs: Dictionary with all remote refs
  393. symrefs: Dictionary with remote symrefs
  394. """
  395. def __init__(self, refs, symrefs) -> None:
  396. """Initialize LsRemoteResult.
  397. Args:
  398. refs: Dictionary with all remote refs
  399. symrefs: Dictionary with remote symrefs
  400. """
  401. self.refs = refs
  402. self.symrefs = symrefs
  403. def _warn_deprecated(self) -> None:
  404. import warnings
  405. warnings.warn(
  406. "Treating LsRemoteResult as a dictionary is deprecated. "
  407. "Use result.refs instead.",
  408. DeprecationWarning,
  409. stacklevel=3,
  410. )
  411. def __eq__(self, other):
  412. """Check equality with another LsRemoteResult."""
  413. if isinstance(other, dict):
  414. self._warn_deprecated()
  415. return self.refs == other
  416. return self.refs == other.refs and self.symrefs == other.symrefs
  417. def __repr__(self) -> str:
  418. """Return string representation of LsRemoteResult."""
  419. return f"{self.__class__.__name__}({self.refs!r}, {self.symrefs!r})"
  420. class SendPackResult(_DeprecatedDictProxy):
  421. """Result of a upload-pack operation.
  422. Attributes:
  423. refs: Dictionary with all remote refs
  424. agent: User agent string
  425. ref_status: Optional dictionary mapping ref name to error message (if it
  426. failed to update), or None if it was updated successfully
  427. """
  428. def __init__(self, refs, agent=None, ref_status=None) -> None:
  429. """Initialize SendPackResult.
  430. Args:
  431. refs: Dictionary with all remote refs
  432. agent: User agent string
  433. ref_status: Optional dictionary mapping ref name to error message
  434. """
  435. self.refs = refs
  436. self.agent = agent
  437. self.ref_status = ref_status
  438. def __eq__(self, other):
  439. """Check equality with another SendPackResult."""
  440. if isinstance(other, dict):
  441. self._warn_deprecated()
  442. return self.refs == other
  443. return self.refs == other.refs and self.agent == other.agent
  444. def __repr__(self) -> str:
  445. """Return string representation of SendPackResult."""
  446. return f"{self.__class__.__name__}({self.refs!r}, {self.agent!r})"
  447. def _read_shallow_updates(pkt_seq):
  448. """Read shallow/unshallow updates from a packet sequence.
  449. Args:
  450. pkt_seq: Sequence of packets
  451. Returns: Tuple of (new_shallow set, new_unshallow set)
  452. """
  453. new_shallow = set()
  454. new_unshallow = set()
  455. for pkt in pkt_seq:
  456. if pkt == b"shallow-info\n": # Git-protocol v2
  457. continue
  458. try:
  459. cmd, sha = pkt.split(b" ", 1)
  460. except ValueError:
  461. raise GitProtocolError(f"unknown command {pkt}")
  462. if cmd == COMMAND_SHALLOW:
  463. new_shallow.add(sha.strip())
  464. elif cmd == COMMAND_UNSHALLOW:
  465. new_unshallow.add(sha.strip())
  466. else:
  467. raise GitProtocolError(f"unknown command {pkt}")
  468. return (new_shallow, new_unshallow)
  469. class _v1ReceivePackHeader:
  470. """Handler for v1 receive-pack header."""
  471. def __init__(self, capabilities, old_refs, new_refs) -> None:
  472. self.want: list[bytes] = []
  473. self.have: list[bytes] = []
  474. self._it = self._handle_receive_pack_head(capabilities, old_refs, new_refs)
  475. self.sent_capabilities = False
  476. def __iter__(self):
  477. """Iterate over the receive-pack header lines."""
  478. return self._it
  479. def _handle_receive_pack_head(self, capabilities, old_refs, new_refs):
  480. """Handle the head of a 'git-receive-pack' request.
  481. Args:
  482. capabilities: List of negotiated capabilities
  483. old_refs: Old refs, as received from the server
  484. new_refs: Refs to change
  485. Returns:
  486. (have, want) tuple
  487. """
  488. self.have = [x for x in old_refs.values() if not x == ZERO_SHA]
  489. for refname in new_refs:
  490. if not isinstance(refname, bytes):
  491. raise TypeError(f"refname is not a bytestring: {refname!r}")
  492. old_sha1 = old_refs.get(refname, ZERO_SHA)
  493. if not isinstance(old_sha1, bytes):
  494. raise TypeError(
  495. f"old sha1 for {refname!r} is not a bytestring: {old_sha1!r}"
  496. )
  497. new_sha1 = new_refs.get(refname, ZERO_SHA)
  498. if not isinstance(new_sha1, bytes):
  499. raise TypeError(
  500. f"old sha1 for {refname!r} is not a bytestring {new_sha1!r}"
  501. )
  502. if old_sha1 != new_sha1:
  503. logger.debug(
  504. "Sending updated ref %r: %r -> %r", refname, old_sha1, new_sha1
  505. )
  506. if self.sent_capabilities:
  507. yield old_sha1 + b" " + new_sha1 + b" " + refname
  508. else:
  509. yield (
  510. old_sha1
  511. + b" "
  512. + new_sha1
  513. + b" "
  514. + refname
  515. + b"\0"
  516. + b" ".join(sorted(capabilities))
  517. )
  518. self.sent_capabilities = True
  519. if new_sha1 not in self.have and new_sha1 != ZERO_SHA:
  520. self.want.append(new_sha1)
  521. yield None
  522. def _read_side_band64k_data(pkt_seq: Iterable[bytes]) -> Iterator[tuple[int, bytes]]:
  523. """Read per-channel data.
  524. This requires the side-band-64k capability.
  525. Args:
  526. pkt_seq: Sequence of packets to read
  527. """
  528. for pkt in pkt_seq:
  529. channel = ord(pkt[:1])
  530. yield channel, pkt[1:]
  531. def find_capability(capabilities, key, value):
  532. """Find a capability in the list of capabilities.
  533. Args:
  534. capabilities: List of capabilities
  535. key: Capability key to search for
  536. value: Optional specific value to match
  537. Returns: The matching capability or None
  538. """
  539. for capability in capabilities:
  540. k, v = parse_capability(capability)
  541. if k != key:
  542. continue
  543. if value and value not in v.split(b" "):
  544. continue
  545. return capability
  546. def _handle_upload_pack_head(
  547. proto,
  548. capabilities,
  549. graph_walker,
  550. wants,
  551. can_read,
  552. depth: Optional[int],
  553. protocol_version,
  554. ):
  555. """Handle the head of a 'git-upload-pack' request.
  556. Args:
  557. proto: Protocol object to read from
  558. capabilities: List of negotiated capabilities
  559. graph_walker: GraphWalker instance to call .ack() on
  560. wants: List of commits to fetch
  561. can_read: function that returns a boolean that indicates
  562. whether there is extra graph data to read on proto
  563. depth: Depth for request
  564. protocol_version: Neogiated Git protocol version.
  565. """
  566. assert isinstance(wants, list) and isinstance(wants[0], bytes)
  567. wantcmd = COMMAND_WANT + b" " + wants[0]
  568. if protocol_version is None:
  569. protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_SEND
  570. if protocol_version != 2:
  571. wantcmd += b" " + b" ".join(sorted(capabilities))
  572. wantcmd += b"\n"
  573. proto.write_pkt_line(wantcmd)
  574. for want in wants[1:]:
  575. proto.write_pkt_line(COMMAND_WANT + b" " + want + b"\n")
  576. if depth not in (0, None) or graph_walker.shallow:
  577. if protocol_version == 2:
  578. if not find_capability(capabilities, CAPABILITY_FETCH, CAPABILITY_SHALLOW):
  579. raise GitProtocolError(
  580. "server does not support shallow capability required for depth"
  581. )
  582. elif CAPABILITY_SHALLOW not in capabilities:
  583. raise GitProtocolError(
  584. "server does not support shallow capability required for depth"
  585. )
  586. for sha in graph_walker.shallow:
  587. proto.write_pkt_line(COMMAND_SHALLOW + b" " + sha + b"\n")
  588. if depth is not None:
  589. proto.write_pkt_line(
  590. COMMAND_DEEPEN + b" " + str(depth).encode("ascii") + b"\n"
  591. )
  592. if protocol_version != 2:
  593. proto.write_pkt_line(None)
  594. have = next(graph_walker)
  595. while have:
  596. proto.write_pkt_line(COMMAND_HAVE + b" " + have + b"\n")
  597. if can_read is not None and can_read():
  598. pkt = proto.read_pkt_line()
  599. parts = pkt.rstrip(b"\n").split(b" ")
  600. if parts[0] == b"ACK":
  601. graph_walker.ack(parts[1])
  602. if parts[2] in (b"continue", b"common"):
  603. pass
  604. elif parts[2] == b"ready":
  605. break
  606. else:
  607. raise AssertionError(
  608. f"{parts[2]} not in ('continue', 'ready', 'common)"
  609. )
  610. have = next(graph_walker)
  611. proto.write_pkt_line(COMMAND_DONE + b"\n")
  612. if protocol_version == 2:
  613. proto.write_pkt_line(None)
  614. if depth not in (0, None):
  615. if can_read is not None:
  616. (new_shallow, new_unshallow) = _read_shallow_updates(proto.read_pkt_seq())
  617. else:
  618. new_shallow = new_unshallow = None
  619. else:
  620. new_shallow = new_unshallow = set()
  621. return (new_shallow, new_unshallow)
  622. def _handle_upload_pack_tail(
  623. proto,
  624. capabilities: set[bytes],
  625. graph_walker,
  626. pack_data: Callable[[bytes], None],
  627. progress: Optional[Callable[[bytes], None]] = None,
  628. rbufsize=_RBUFSIZE,
  629. protocol_version=0,
  630. ) -> None:
  631. """Handle the tail of a 'git-upload-pack' request.
  632. Args:
  633. proto: Protocol object to read from
  634. capabilities: List of negotiated capabilities
  635. graph_walker: GraphWalker instance to call .ack() on
  636. pack_data: Function to call with pack data
  637. progress: Optional progress reporting function
  638. rbufsize: Read buffer size
  639. protocol_version: Neogiated Git protocol version.
  640. """
  641. pkt = proto.read_pkt_line()
  642. while pkt:
  643. parts = pkt.rstrip(b"\n").split(b" ")
  644. if protocol_version == 2 and parts[0] != "packfile":
  645. break
  646. else:
  647. if parts[0] == b"ACK":
  648. graph_walker.ack(parts[1])
  649. if parts[0] == b"NAK":
  650. graph_walker.nak()
  651. if len(parts) < 3 or parts[2] not in (
  652. b"ready",
  653. b"continue",
  654. b"common",
  655. ):
  656. break
  657. pkt = proto.read_pkt_line()
  658. if CAPABILITY_SIDE_BAND_64K in capabilities or protocol_version == 2:
  659. if progress is None:
  660. # Just ignore progress data
  661. def progress(x) -> None:
  662. pass
  663. for chan, data in _read_side_band64k_data(proto.read_pkt_seq()):
  664. if chan == SIDE_BAND_CHANNEL_DATA:
  665. pack_data(data)
  666. elif chan == SIDE_BAND_CHANNEL_PROGRESS:
  667. progress(data)
  668. else:
  669. raise AssertionError(f"Invalid sideband channel {chan}")
  670. else:
  671. while True:
  672. data = proto.read(rbufsize)
  673. if data == b"":
  674. break
  675. pack_data(data)
  676. def _extract_symrefs_and_agent(capabilities):
  677. """Extract symrefs and agent from capabilities.
  678. Args:
  679. capabilities: List of capabilities
  680. Returns:
  681. (symrefs, agent) tuple
  682. """
  683. symrefs = {}
  684. agent = None
  685. for capability in capabilities:
  686. k, v = parse_capability(capability)
  687. if k == CAPABILITY_SYMREF:
  688. (src, dst) = v.split(b":", 1)
  689. symrefs[src] = dst
  690. if k == CAPABILITY_AGENT:
  691. agent = v
  692. return (symrefs, agent)
  693. # TODO(durin42): this doesn't correctly degrade if the server doesn't
  694. # support some capabilities. This should work properly with servers
  695. # that don't support multi_ack.
  696. class GitClient:
  697. """Git smart server client."""
  698. def __init__(
  699. self,
  700. thin_packs=True,
  701. report_activity=None,
  702. quiet=False,
  703. include_tags=False,
  704. **kwargs,
  705. ) -> None:
  706. """Create a new GitClient instance.
  707. Args:
  708. thin_packs: Whether or not thin packs should be retrieved
  709. report_activity: Optional callback for reporting transport
  710. activity.
  711. include_tags: send annotated tags when sending the objects they point
  712. to
  713. """
  714. self._report_activity = report_activity
  715. self._report_status_parser: Optional[ReportStatusParser] = None
  716. self._fetch_capabilities = set(UPLOAD_CAPABILITIES)
  717. self._fetch_capabilities.add(capability_agent())
  718. self._send_capabilities = set(RECEIVE_CAPABILITIES)
  719. self._send_capabilities.add(capability_agent())
  720. if quiet:
  721. self._send_capabilities.add(CAPABILITY_QUIET)
  722. if not thin_packs:
  723. self._fetch_capabilities.remove(CAPABILITY_THIN_PACK)
  724. if include_tags:
  725. self._fetch_capabilities.add(CAPABILITY_INCLUDE_TAG)
  726. self.protocol_version = 0 # will be overridden later
  727. def get_url(self, path) -> str:
  728. """Retrieves full url to given path.
  729. Args:
  730. path: Repository path (as string)
  731. Returns:
  732. Url to path (as string)
  733. """
  734. raise NotImplementedError(self.get_url)
  735. @classmethod
  736. def from_parsedurl(cls, parsedurl, **kwargs) -> "GitClient":
  737. """Create an instance of this client from a urlparse.parsed object.
  738. Args:
  739. parsedurl: Result of urlparse()
  740. Returns:
  741. A `GitClient` object
  742. """
  743. raise NotImplementedError(cls.from_parsedurl)
  744. def send_pack(
  745. self,
  746. path: str,
  747. update_refs,
  748. generate_pack_data: Callable[
  749. [set[bytes], set[bytes], bool], tuple[int, Iterator[UnpackedObject]]
  750. ],
  751. progress=None,
  752. ) -> SendPackResult:
  753. """Upload a pack to a remote repository.
  754. Args:
  755. path: Repository path (as bytestring)
  756. update_refs: Function to determine changes to remote refs. Receive
  757. dict with existing remote refs, returns dict with
  758. changed refs (name -> sha, where sha=ZERO_SHA for deletions)
  759. generate_pack_data: Function that can return a tuple
  760. with number of objects and list of pack data to include
  761. progress: Optional progress function
  762. Returns:
  763. SendPackResult object
  764. Raises:
  765. SendPackError: if server rejects the pack data
  766. """
  767. raise NotImplementedError(self.send_pack)
  768. def clone(
  769. self,
  770. path,
  771. target_path,
  772. mkdir: bool = True,
  773. bare: bool = False,
  774. origin: Optional[str] = "origin",
  775. checkout=None,
  776. branch=None,
  777. progress=None,
  778. depth: Optional[int] = None,
  779. ref_prefix: Optional[list[Ref]] = None,
  780. filter_spec=None,
  781. protocol_version: Optional[int] = None,
  782. ) -> Repo:
  783. """Clone a repository."""
  784. if mkdir:
  785. os.mkdir(target_path)
  786. try:
  787. target = None
  788. if not bare:
  789. target = Repo.init(target_path)
  790. if checkout is None:
  791. checkout = True
  792. else:
  793. if checkout:
  794. raise ValueError("checkout and bare are incompatible")
  795. target = Repo.init_bare(target_path)
  796. # TODO(jelmer): abstract method for get_location?
  797. if isinstance(self, (LocalGitClient, SubprocessGitClient)):
  798. encoded_path = path.encode("utf-8")
  799. else:
  800. encoded_path = self.get_url(path).encode("utf-8")
  801. assert target is not None
  802. if origin is not None:
  803. target_config = target.get_config()
  804. target_config.set(
  805. (b"remote", origin.encode("utf-8")), b"url", encoded_path
  806. )
  807. target_config.set(
  808. (b"remote", origin.encode("utf-8")),
  809. b"fetch",
  810. b"+refs/heads/*:refs/remotes/" + origin.encode("utf-8") + b"/*",
  811. )
  812. target_config.write_to_path()
  813. ref_message = b"clone: from " + encoded_path
  814. result = self.fetch(
  815. path,
  816. target,
  817. progress=progress,
  818. depth=depth,
  819. ref_prefix=ref_prefix,
  820. filter_spec=filter_spec,
  821. protocol_version=protocol_version,
  822. )
  823. if origin is not None:
  824. _import_remote_refs(
  825. target.refs, origin, result.refs, message=ref_message
  826. )
  827. origin_head = result.symrefs.get(b"HEAD")
  828. origin_sha = result.refs.get(b"HEAD")
  829. if origin is None or (origin_sha and not origin_head):
  830. # set detached HEAD
  831. target.refs[b"HEAD"] = origin_sha
  832. head = origin_sha
  833. else:
  834. _set_origin_head(target.refs, origin.encode("utf-8"), origin_head)
  835. head_ref = _set_default_branch(
  836. target.refs,
  837. origin.encode("utf-8"),
  838. origin_head,
  839. branch,
  840. ref_message,
  841. )
  842. # Update target head
  843. if head_ref:
  844. head = _set_head(target.refs, head_ref, ref_message)
  845. else:
  846. head = None
  847. if checkout and head is not None:
  848. target.get_worktree().reset_index()
  849. except BaseException:
  850. if target is not None:
  851. target.close()
  852. if mkdir:
  853. import shutil
  854. shutil.rmtree(target_path)
  855. raise
  856. return target
  857. def fetch(
  858. self,
  859. path: str,
  860. target: BaseRepo,
  861. determine_wants: Optional[
  862. Callable[[dict[bytes, bytes], Optional[int]], list[bytes]]
  863. ] = None,
  864. progress: Optional[Callable[[bytes], None]] = None,
  865. depth: Optional[int] = None,
  866. ref_prefix: Optional[list[Ref]] = None,
  867. filter_spec: Optional[bytes] = None,
  868. protocol_version: Optional[int] = None,
  869. ) -> FetchPackResult:
  870. """Fetch into a target repository.
  871. Args:
  872. path: Path to fetch from (as bytestring)
  873. target: Target repository to fetch into
  874. determine_wants: Optional function to determine what refs to fetch.
  875. Receives dictionary of name->sha, should return
  876. list of shas to fetch. Defaults to all shas.
  877. progress: Optional progress function
  878. depth: Depth to fetch at
  879. ref_prefix: List of prefixes of desired references, as a list of
  880. bytestrings. Filtering is done by the server if supported, and
  881. client side otherwise.
  882. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  883. Only used if the server supports the Git protocol-v2 'filter'
  884. feature, and ignored otherwise.
  885. protocol_version: Desired Git protocol version. By default the highest
  886. mutually supported protocol version will be used.
  887. Returns:
  888. Dictionary with all remote refs (not just those fetched)
  889. """
  890. if determine_wants is None:
  891. determine_wants = target.object_store.determine_wants_all
  892. if CAPABILITY_THIN_PACK in self._fetch_capabilities:
  893. from tempfile import SpooledTemporaryFile
  894. f: IO[bytes] = SpooledTemporaryFile(
  895. max_size=PACK_SPOOL_FILE_MAX_SIZE,
  896. prefix="incoming-",
  897. dir=getattr(target.object_store, "path", None),
  898. )
  899. def commit() -> None:
  900. if f.tell():
  901. f.seek(0)
  902. target.object_store.add_thin_pack(f.read, None, progress=progress) # type: ignore
  903. f.close()
  904. def abort() -> None:
  905. f.close()
  906. else:
  907. f, commit, abort = target.object_store.add_pack()
  908. try:
  909. result = self.fetch_pack(
  910. path,
  911. determine_wants,
  912. target.get_graph_walker(),
  913. f.write,
  914. progress=progress,
  915. depth=depth,
  916. ref_prefix=ref_prefix,
  917. filter_spec=filter_spec,
  918. protocol_version=protocol_version,
  919. )
  920. except BaseException:
  921. abort()
  922. raise
  923. else:
  924. commit()
  925. target.update_shallow(result.new_shallow, result.new_unshallow)
  926. return result
  927. def fetch_pack(
  928. self,
  929. path: str,
  930. determine_wants,
  931. graph_walker,
  932. pack_data,
  933. *,
  934. progress: Optional[Callable[[bytes], None]] = None,
  935. depth: Optional[int] = None,
  936. ref_prefix: Optional[list[Ref]] = None,
  937. filter_spec=None,
  938. protocol_version: Optional[int] = None,
  939. ) -> FetchPackResult:
  940. """Retrieve a pack from a git smart server.
  941. Args:
  942. path: Remote path to fetch from
  943. determine_wants: Function determine what refs
  944. to fetch. Receives dictionary of name->sha, should return
  945. list of shas to fetch.
  946. graph_walker: Object with next() and ack().
  947. pack_data: Callback called for each bit of data in the pack
  948. progress: Callback for progress reports (strings)
  949. depth: Shallow fetch depth
  950. ref_prefix: List of prefixes of desired references, as a list of
  951. bytestrings. Filtering is done by the server if supported, and
  952. client side otherwise.
  953. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  954. Only used if the server supports the Git protocol-v2 'filter'
  955. feature, and ignored otherwise.
  956. protocol_version: Desired Git protocol version. By default the highest
  957. mutually supported protocol version will be used.
  958. Returns:
  959. FetchPackResult object
  960. """
  961. raise NotImplementedError(self.fetch_pack)
  962. def get_refs(
  963. self,
  964. path,
  965. protocol_version: Optional[int] = None,
  966. ref_prefix: Optional[list[Ref]] = None,
  967. ) -> LsRemoteResult:
  968. """Retrieve the current refs from a git smart server.
  969. Args:
  970. path: Path to the repo to fetch from. (as bytestring)
  971. protocol_version: Desired Git protocol version.
  972. ref_prefix: Prefix filter for refs.
  973. Returns:
  974. LsRemoteResult object with refs and symrefs
  975. """
  976. raise NotImplementedError(self.get_refs)
  977. @staticmethod
  978. def _should_send_pack(new_refs):
  979. # The packfile MUST NOT be sent if the only command used is delete.
  980. return any(sha != ZERO_SHA for sha in new_refs.values())
  981. def _negotiate_receive_pack_capabilities(self, server_capabilities):
  982. negotiated_capabilities = self._send_capabilities & server_capabilities
  983. (agent, _symrefs) = _extract_symrefs_and_agent(server_capabilities)
  984. (extract_capability_names(server_capabilities) - KNOWN_RECEIVE_CAPABILITIES)
  985. # TODO(jelmer): warn about unknown capabilities
  986. return negotiated_capabilities, agent
  987. def _handle_receive_pack_tail(
  988. self,
  989. proto: Protocol,
  990. capabilities: set[bytes],
  991. progress: Optional[Callable[[bytes], None]] = None,
  992. ) -> Optional[dict[bytes, Optional[str]]]:
  993. """Handle the tail of a 'git-receive-pack' request.
  994. Args:
  995. proto: Protocol object to read from
  996. capabilities: List of negotiated capabilities
  997. progress: Optional progress reporting function
  998. Returns:
  999. dict mapping ref name to:
  1000. error message if the ref failed to update
  1001. None if it was updated successfully
  1002. """
  1003. if CAPABILITY_SIDE_BAND_64K in capabilities or self.protocol_version == 2:
  1004. if progress is None:
  1005. def progress(x) -> None:
  1006. pass
  1007. if CAPABILITY_REPORT_STATUS in capabilities:
  1008. assert self._report_status_parser is not None
  1009. pktline_parser = PktLineParser(self._report_status_parser.handle_packet)
  1010. for chan, data in _read_side_band64k_data(proto.read_pkt_seq()):
  1011. if chan == SIDE_BAND_CHANNEL_DATA:
  1012. if CAPABILITY_REPORT_STATUS in capabilities:
  1013. pktline_parser.parse(data)
  1014. elif chan == SIDE_BAND_CHANNEL_PROGRESS:
  1015. progress(data)
  1016. else:
  1017. raise AssertionError(f"Invalid sideband channel {chan}")
  1018. else:
  1019. if CAPABILITY_REPORT_STATUS in capabilities:
  1020. assert self._report_status_parser
  1021. for pkt in proto.read_pkt_seq():
  1022. self._report_status_parser.handle_packet(pkt)
  1023. if self._report_status_parser is not None:
  1024. return dict(self._report_status_parser.check())
  1025. return None
  1026. def _negotiate_upload_pack_capabilities(self, server_capabilities):
  1027. (extract_capability_names(server_capabilities) - KNOWN_UPLOAD_CAPABILITIES)
  1028. # TODO(jelmer): warn about unknown capabilities
  1029. fetch_capa = None
  1030. for capability in server_capabilities:
  1031. k, v = parse_capability(capability)
  1032. if self.protocol_version == 2 and k == CAPABILITY_FETCH:
  1033. fetch_capa = CAPABILITY_FETCH
  1034. fetch_features = []
  1035. v = v.strip().split(b" ")
  1036. if b"shallow" in v:
  1037. fetch_features.append(CAPABILITY_SHALLOW)
  1038. if b"filter" in v:
  1039. fetch_features.append(CAPABILITY_FILTER)
  1040. for i in range(len(fetch_features)):
  1041. if i == 0:
  1042. fetch_capa += b"="
  1043. else:
  1044. fetch_capa += b" "
  1045. fetch_capa += fetch_features[i]
  1046. (symrefs, agent) = _extract_symrefs_and_agent(server_capabilities)
  1047. negotiated_capabilities = self._fetch_capabilities & server_capabilities
  1048. if fetch_capa:
  1049. negotiated_capabilities.add(fetch_capa)
  1050. return (negotiated_capabilities, symrefs, agent)
  1051. def archive(
  1052. self,
  1053. path,
  1054. committish,
  1055. write_data,
  1056. progress=None,
  1057. write_error=None,
  1058. format=None,
  1059. subdirs=None,
  1060. prefix=None,
  1061. ) -> None:
  1062. """Retrieve an archive of the specified tree."""
  1063. raise NotImplementedError(self.archive)
  1064. @staticmethod
  1065. def _warn_filter_objects() -> None:
  1066. import warnings
  1067. warnings.warn(
  1068. "object filtering not recognized by server, ignoring",
  1069. UserWarning,
  1070. )
  1071. def check_wants(wants, refs) -> None:
  1072. """Check that a set of wants is valid.
  1073. Args:
  1074. wants: Set of object SHAs to fetch
  1075. refs: Refs dictionary to check against
  1076. """
  1077. missing = set(wants) - {
  1078. v for (k, v) in refs.items() if not k.endswith(PEELED_TAG_SUFFIX)
  1079. }
  1080. if missing:
  1081. raise InvalidWants(missing)
  1082. def _remote_error_from_stderr(stderr):
  1083. if stderr is None:
  1084. return HangupException()
  1085. lines = [line.rstrip(b"\n") for line in stderr.readlines()]
  1086. for line in lines:
  1087. if line.startswith(b"ERROR: "):
  1088. return GitProtocolError(line[len(b"ERROR: ") :].decode("utf-8", "replace"))
  1089. return HangupException(lines)
  1090. class TraditionalGitClient(GitClient):
  1091. """Traditional Git client."""
  1092. DEFAULT_ENCODING = "utf-8"
  1093. def __init__(self, path_encoding=DEFAULT_ENCODING, **kwargs) -> None:
  1094. self._remote_path_encoding = path_encoding
  1095. super().__init__(**kwargs)
  1096. def _connect(
  1097. self,
  1098. cmd: bytes,
  1099. path: Union[str, bytes],
  1100. protocol_version: Optional[int] = None,
  1101. ) -> tuple[Protocol, Callable[[], bool], Optional[IO[bytes]]]:
  1102. """Create a connection to the server.
  1103. This method is abstract - concrete implementations should
  1104. implement their own variant which connects to the server and
  1105. returns an initialized Protocol object with the service ready
  1106. for use and a can_read function which may be used to see if
  1107. reads would block.
  1108. Args:
  1109. cmd: The git service name to which we should connect.
  1110. path: The path we should pass to the service. (as bytestirng)
  1111. protocol_version: Desired Git protocol version. By default the highest
  1112. mutually supported protocol version will be used.
  1113. """
  1114. raise NotImplementedError
  1115. def send_pack(self, path, update_refs, generate_pack_data, progress=None):
  1116. """Upload a pack to a remote repository.
  1117. Args:
  1118. path: Repository path (as bytestring)
  1119. update_refs: Function to determine changes to remote refs.
  1120. Receive dict with existing remote refs, returns dict with
  1121. changed refs (name -> sha, where sha=ZERO_SHA for deletions)
  1122. generate_pack_data: Function that can return a tuple with
  1123. number of objects and pack data to upload.
  1124. progress: Optional callback called with progress updates
  1125. Returns:
  1126. SendPackResult
  1127. Raises:
  1128. SendPackError: if server rejects the pack data
  1129. """
  1130. self.protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_SEND
  1131. proto, unused_can_read, stderr = self._connect(b"receive-pack", path)
  1132. with proto:
  1133. try:
  1134. old_refs, server_capabilities = read_pkt_refs_v1(proto.read_pkt_seq())
  1135. except HangupException as exc:
  1136. raise _remote_error_from_stderr(stderr) from exc
  1137. (
  1138. negotiated_capabilities,
  1139. agent,
  1140. ) = self._negotiate_receive_pack_capabilities(server_capabilities)
  1141. if CAPABILITY_REPORT_STATUS in negotiated_capabilities:
  1142. self._report_status_parser = ReportStatusParser()
  1143. report_status_parser = self._report_status_parser
  1144. try:
  1145. new_refs = orig_new_refs = update_refs(dict(old_refs))
  1146. except BaseException:
  1147. proto.write_pkt_line(None)
  1148. raise
  1149. if set(new_refs.items()).issubset(set(old_refs.items())):
  1150. proto.write_pkt_line(None)
  1151. return SendPackResult(new_refs, agent=agent, ref_status={})
  1152. if CAPABILITY_DELETE_REFS not in server_capabilities:
  1153. # Server does not support deletions. Fail later.
  1154. new_refs = dict(orig_new_refs)
  1155. for ref, sha in orig_new_refs.items():
  1156. if sha == ZERO_SHA:
  1157. if CAPABILITY_REPORT_STATUS in negotiated_capabilities:
  1158. report_status_parser._ref_statuses.append(
  1159. b"ng " + ref + b" remote does not support deleting refs"
  1160. )
  1161. report_status_parser._ref_status_ok = False
  1162. del new_refs[ref]
  1163. if new_refs is None:
  1164. proto.write_pkt_line(None)
  1165. return SendPackResult(old_refs, agent=agent, ref_status={})
  1166. if len(new_refs) == 0 and orig_new_refs:
  1167. # NOOP - Original new refs filtered out by policy
  1168. proto.write_pkt_line(None)
  1169. if report_status_parser is not None:
  1170. ref_status = dict(report_status_parser.check())
  1171. else:
  1172. ref_status = None
  1173. return SendPackResult(old_refs, agent=agent, ref_status=ref_status)
  1174. header_handler = _v1ReceivePackHeader(
  1175. negotiated_capabilities, old_refs, new_refs
  1176. )
  1177. for pkt in header_handler:
  1178. proto.write_pkt_line(pkt)
  1179. pack_data_count, pack_data = generate_pack_data(
  1180. header_handler.have,
  1181. header_handler.want,
  1182. ofs_delta=(CAPABILITY_OFS_DELTA in negotiated_capabilities),
  1183. progress=progress,
  1184. )
  1185. if self._should_send_pack(new_refs):
  1186. for chunk in PackChunkGenerator(
  1187. pack_data_count, pack_data, progress=progress
  1188. ):
  1189. proto.write(chunk)
  1190. ref_status = self._handle_receive_pack_tail(
  1191. proto, negotiated_capabilities, progress
  1192. )
  1193. return SendPackResult(new_refs, agent=agent, ref_status=ref_status)
  1194. def fetch_pack(
  1195. self,
  1196. path,
  1197. determine_wants,
  1198. graph_walker,
  1199. pack_data,
  1200. progress=None,
  1201. depth: Optional[int] = None,
  1202. ref_prefix: Optional[list[Ref]] = None,
  1203. filter_spec=None,
  1204. protocol_version: Optional[int] = None,
  1205. ):
  1206. """Retrieve a pack from a git smart server.
  1207. Args:
  1208. path: Remote path to fetch from
  1209. determine_wants: Function determine what refs
  1210. to fetch. Receives dictionary of name->sha, should return
  1211. list of shas to fetch.
  1212. graph_walker: Object with next() and ack().
  1213. pack_data: Callback called for each bit of data in the pack
  1214. progress: Callback for progress reports (strings)
  1215. depth: Shallow fetch depth
  1216. ref_prefix: List of prefixes of desired references, as a list of
  1217. bytestrings. Filtering is done by the server if supported, and
  1218. client side otherwise.
  1219. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  1220. Only used if the server supports the Git protocol-v2 'filter'
  1221. feature, and ignored otherwise.
  1222. protocol_version: Desired Git protocol version. By default the highest
  1223. mutually supported protocol version will be used.
  1224. Returns:
  1225. FetchPackResult object
  1226. """
  1227. if (
  1228. protocol_version is not None
  1229. and protocol_version not in GIT_PROTOCOL_VERSIONS
  1230. ):
  1231. raise ValueError(f"unknown Git protocol version {protocol_version}")
  1232. proto, can_read, stderr = self._connect(b"upload-pack", path, protocol_version)
  1233. server_protocol_version = negotiate_protocol_version(proto)
  1234. if server_protocol_version not in GIT_PROTOCOL_VERSIONS:
  1235. raise ValueError(
  1236. f"unknown Git protocol version {server_protocol_version} used by server"
  1237. )
  1238. if protocol_version and server_protocol_version > protocol_version:
  1239. raise ValueError(
  1240. f"bad Git protocol version {server_protocol_version} used by server"
  1241. )
  1242. self.protocol_version = server_protocol_version
  1243. with proto:
  1244. if self.protocol_version == 2:
  1245. try:
  1246. server_capabilities = read_server_capabilities(proto.read_pkt_seq())
  1247. except HangupException as exc:
  1248. raise _remote_error_from_stderr(stderr) from exc
  1249. (
  1250. negotiated_capabilities,
  1251. symrefs,
  1252. agent,
  1253. ) = self._negotiate_upload_pack_capabilities(server_capabilities)
  1254. proto.write_pkt_line(b"command=ls-refs\n")
  1255. proto.write(b"0001") # delim-pkt
  1256. proto.write_pkt_line(b"symrefs")
  1257. proto.write_pkt_line(b"peel")
  1258. if ref_prefix is None:
  1259. ref_prefix = DEFAULT_REF_PREFIX
  1260. for prefix in ref_prefix:
  1261. proto.write_pkt_line(b"ref-prefix " + prefix)
  1262. proto.write_pkt_line(None)
  1263. refs, symrefs, _peeled = read_pkt_refs_v2(proto.read_pkt_seq())
  1264. else:
  1265. try:
  1266. refs, server_capabilities = read_pkt_refs_v1(proto.read_pkt_seq())
  1267. except HangupException as exc:
  1268. raise _remote_error_from_stderr(stderr) from exc
  1269. (
  1270. negotiated_capabilities,
  1271. symrefs,
  1272. agent,
  1273. ) = self._negotiate_upload_pack_capabilities(server_capabilities)
  1274. if ref_prefix is not None:
  1275. refs = filter_ref_prefix(refs, ref_prefix)
  1276. if refs is None:
  1277. proto.write_pkt_line(None)
  1278. return FetchPackResult(refs, symrefs, agent)
  1279. try:
  1280. if depth is not None:
  1281. wants = determine_wants(refs, depth=depth)
  1282. else:
  1283. wants = determine_wants(refs)
  1284. except BaseException:
  1285. proto.write_pkt_line(None)
  1286. raise
  1287. if wants is not None:
  1288. wants = [cid for cid in wants if cid != ZERO_SHA]
  1289. if not wants:
  1290. proto.write_pkt_line(None)
  1291. return FetchPackResult(refs, symrefs, agent)
  1292. if self.protocol_version == 2:
  1293. proto.write_pkt_line(b"command=fetch\n")
  1294. proto.write(b"0001") # delim-pkt
  1295. if CAPABILITY_THIN_PACK in self._fetch_capabilities:
  1296. proto.write(pkt_line(b"thin-pack\n"))
  1297. if (
  1298. find_capability(
  1299. negotiated_capabilities, CAPABILITY_FETCH, CAPABILITY_FILTER
  1300. )
  1301. and filter_spec
  1302. ):
  1303. proto.write(pkt_line(b"filter %s\n" % filter_spec))
  1304. elif filter_spec:
  1305. self._warn_filter_objects()
  1306. elif filter_spec:
  1307. self._warn_filter_objects()
  1308. (new_shallow, new_unshallow) = _handle_upload_pack_head(
  1309. proto,
  1310. negotiated_capabilities,
  1311. graph_walker,
  1312. wants,
  1313. can_read,
  1314. depth=depth,
  1315. protocol_version=self.protocol_version,
  1316. )
  1317. _handle_upload_pack_tail(
  1318. proto,
  1319. negotiated_capabilities,
  1320. graph_walker,
  1321. pack_data,
  1322. progress,
  1323. protocol_version=self.protocol_version,
  1324. )
  1325. return FetchPackResult(refs, symrefs, agent, new_shallow, new_unshallow)
  1326. def get_refs(
  1327. self,
  1328. path,
  1329. protocol_version: Optional[int] = None,
  1330. ref_prefix: Optional[list[Ref]] = None,
  1331. ):
  1332. """Retrieve the current refs from a git smart server."""
  1333. # stock `git ls-remote` uses upload-pack
  1334. if (
  1335. protocol_version is not None
  1336. and protocol_version not in GIT_PROTOCOL_VERSIONS
  1337. ):
  1338. raise ValueError(f"unknown Git protocol version {protocol_version}")
  1339. proto, _, stderr = self._connect(b"upload-pack", path, protocol_version)
  1340. server_protocol_version = negotiate_protocol_version(proto)
  1341. if server_protocol_version not in GIT_PROTOCOL_VERSIONS:
  1342. raise ValueError(
  1343. f"unknown Git protocol version {server_protocol_version} used by server"
  1344. )
  1345. if protocol_version and server_protocol_version > protocol_version:
  1346. raise ValueError(
  1347. f"bad Git protocol version {server_protocol_version} used by server"
  1348. )
  1349. self.protocol_version = server_protocol_version
  1350. if self.protocol_version == 2:
  1351. server_capabilities = read_server_capabilities(proto.read_pkt_seq())
  1352. proto.write_pkt_line(b"command=ls-refs\n")
  1353. proto.write(b"0001") # delim-pkt
  1354. proto.write_pkt_line(b"symrefs")
  1355. proto.write_pkt_line(b"peel")
  1356. if ref_prefix is None:
  1357. ref_prefix = DEFAULT_REF_PREFIX
  1358. for prefix in ref_prefix:
  1359. proto.write_pkt_line(b"ref-prefix " + prefix)
  1360. proto.write_pkt_line(None)
  1361. with proto:
  1362. try:
  1363. refs, symrefs, peeled = read_pkt_refs_v2(proto.read_pkt_seq())
  1364. except HangupException as exc:
  1365. raise _remote_error_from_stderr(stderr) from exc
  1366. proto.write_pkt_line(None)
  1367. for refname, refvalue in peeled.items():
  1368. refs[refname + PEELED_TAG_SUFFIX] = refvalue
  1369. return LsRemoteResult(refs, symrefs)
  1370. else:
  1371. with proto:
  1372. try:
  1373. refs, server_capabilities = read_pkt_refs_v1(proto.read_pkt_seq())
  1374. except HangupException as exc:
  1375. raise _remote_error_from_stderr(stderr) from exc
  1376. proto.write_pkt_line(None)
  1377. (symrefs, _agent) = _extract_symrefs_and_agent(server_capabilities)
  1378. if ref_prefix is not None:
  1379. refs = filter_ref_prefix(refs, ref_prefix)
  1380. return LsRemoteResult(refs, symrefs)
  1381. def archive(
  1382. self,
  1383. path,
  1384. committish,
  1385. write_data,
  1386. progress=None,
  1387. write_error=None,
  1388. format=None,
  1389. subdirs=None,
  1390. prefix=None,
  1391. ) -> None:
  1392. """Request an archive of a specific commit.
  1393. Args:
  1394. path: Repository path
  1395. committish: Commit ID or ref to archive
  1396. write_data: Function to write archive data
  1397. progress: Optional progress callback
  1398. write_error: Optional error callback
  1399. format: Optional archive format
  1400. subdirs: Optional subdirectories to include
  1401. prefix: Optional prefix for archived files
  1402. """
  1403. proto, can_read, stderr = self._connect(b"upload-archive", path)
  1404. with proto:
  1405. if format is not None:
  1406. proto.write_pkt_line(b"argument --format=" + format)
  1407. proto.write_pkt_line(b"argument " + committish)
  1408. if subdirs is not None:
  1409. for subdir in subdirs:
  1410. proto.write_pkt_line(b"argument " + subdir)
  1411. if prefix is not None:
  1412. proto.write_pkt_line(b"argument --prefix=" + prefix)
  1413. proto.write_pkt_line(None)
  1414. try:
  1415. pkt = proto.read_pkt_line()
  1416. except HangupException as exc:
  1417. raise _remote_error_from_stderr(stderr) from exc
  1418. if pkt == b"NACK\n" or pkt == b"NACK":
  1419. return
  1420. elif pkt == b"ACK\n" or pkt == b"ACK":
  1421. pass
  1422. elif pkt and pkt.startswith(b"ERR "):
  1423. raise GitProtocolError(pkt[4:].rstrip(b"\n").decode("utf-8", "replace"))
  1424. else:
  1425. raise AssertionError(f"invalid response {pkt!r}")
  1426. ret = proto.read_pkt_line()
  1427. if ret is not None:
  1428. raise AssertionError("expected pkt tail")
  1429. for chan, data in _read_side_band64k_data(proto.read_pkt_seq()):
  1430. if chan == SIDE_BAND_CHANNEL_DATA:
  1431. write_data(data)
  1432. elif chan == SIDE_BAND_CHANNEL_PROGRESS:
  1433. progress(data)
  1434. elif chan == SIDE_BAND_CHANNEL_FATAL:
  1435. write_error(data)
  1436. else:
  1437. raise AssertionError(f"Invalid sideband channel {chan}")
  1438. class TCPGitClient(TraditionalGitClient):
  1439. """A Git Client that works over TCP directly (i.e. git://)."""
  1440. def __init__(self, host, port=None, **kwargs) -> None:
  1441. """Initialize a TCPGitClient.
  1442. Args:
  1443. host: Hostname or IP address to connect to
  1444. port: Port number (defaults to TCP_GIT_PORT)
  1445. **kwargs: Additional arguments for GitClient
  1446. """
  1447. if port is None:
  1448. port = TCP_GIT_PORT
  1449. self._host = host
  1450. self._port = port
  1451. super().__init__(**kwargs)
  1452. @classmethod
  1453. def from_parsedurl(cls, parsedurl, **kwargs):
  1454. """Create an instance of TCPGitClient from a parsed URL.
  1455. Args:
  1456. parsedurl: Result of urlparse()
  1457. **kwargs: Additional arguments for the client
  1458. Returns:
  1459. A TCPGitClient instance
  1460. """
  1461. return cls(parsedurl.hostname, port=parsedurl.port, **kwargs)
  1462. def get_url(self, path):
  1463. """Get the URL for a TCP git connection.
  1464. Args:
  1465. path: Repository path
  1466. Returns:
  1467. git:// URL for the path
  1468. """
  1469. netloc = self._host
  1470. if self._port is not None and self._port != TCP_GIT_PORT:
  1471. netloc += f":{self._port}"
  1472. return urlunsplit(("git", netloc, path, "", ""))
  1473. def _connect(
  1474. self,
  1475. cmd: bytes,
  1476. path: Union[str, bytes],
  1477. protocol_version: Optional[int] = None,
  1478. ) -> tuple[Protocol, Callable[[], bool], Optional[IO[bytes]]]:
  1479. if not isinstance(cmd, bytes):
  1480. raise TypeError(cmd)
  1481. if not isinstance(path, bytes):
  1482. path = path.encode(self._remote_path_encoding)
  1483. sockaddrs = socket.getaddrinfo(
  1484. self._host, self._port, socket.AF_UNSPEC, socket.SOCK_STREAM
  1485. )
  1486. s = None
  1487. err = OSError(f"no address found for {self._host}")
  1488. for family, socktype, protof, canonname, sockaddr in sockaddrs:
  1489. s = socket.socket(family, socktype, protof)
  1490. s.setsockopt(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)
  1491. try:
  1492. s.connect(sockaddr)
  1493. break
  1494. except OSError as e:
  1495. err = e
  1496. if s is not None:
  1497. s.close()
  1498. s = None
  1499. if s is None:
  1500. raise err
  1501. # -1 means system default buffering
  1502. rfile = s.makefile("rb", -1)
  1503. # 0 means unbuffered
  1504. wfile = s.makefile("wb", 0)
  1505. def close() -> None:
  1506. rfile.close()
  1507. wfile.close()
  1508. s.close()
  1509. proto = Protocol(
  1510. rfile.read,
  1511. wfile.write,
  1512. close,
  1513. report_activity=self._report_activity,
  1514. )
  1515. if path.startswith(b"/~"):
  1516. path = path[1:]
  1517. if cmd == b"upload-pack":
  1518. if protocol_version is None:
  1519. self.protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_FETCH
  1520. else:
  1521. self.protocol_version = protocol_version
  1522. else:
  1523. self.protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_SEND
  1524. if cmd == b"upload-pack" and self.protocol_version == 2:
  1525. # Git protocol version advertisement is hidden behind two NUL bytes
  1526. # for compatibility with older Git server implementations, which
  1527. # would crash if something other than a "host=" header was found
  1528. # after the first NUL byte.
  1529. version_str = b"\0\0version=%d\0" % self.protocol_version
  1530. else:
  1531. version_str = b""
  1532. # TODO(jelmer): Alternative to ascii?
  1533. proto.send_cmd(
  1534. b"git-" + cmd, path, b"host=" + self._host.encode("ascii") + version_str
  1535. )
  1536. return proto, lambda: _fileno_can_read(s), None
  1537. class SubprocessWrapper:
  1538. """A socket-like object that talks to a subprocess via pipes."""
  1539. def __init__(self, proc) -> None:
  1540. """Initialize a SubprocessWrapper.
  1541. Args:
  1542. proc: Subprocess.Popen instance to wrap
  1543. """
  1544. self.proc = proc
  1545. self.read = BufferedReader(proc.stdout).read
  1546. self.write = proc.stdin.write
  1547. @property
  1548. def stderr(self):
  1549. """Return the stderr stream of the subprocess."""
  1550. return self.proc.stderr
  1551. def can_read(self):
  1552. """Check if there is data available to read.
  1553. Returns: True if data is available, False otherwise
  1554. """
  1555. if sys.platform == "win32":
  1556. from msvcrt import get_osfhandle
  1557. handle = get_osfhandle(self.proc.stdout.fileno())
  1558. return _win32_peek_avail(handle) != 0
  1559. else:
  1560. return _fileno_can_read(self.proc.stdout.fileno())
  1561. def close(self, timeout: Optional[int] = 60) -> None:
  1562. """Close the subprocess and wait for it to terminate.
  1563. Args:
  1564. timeout: Maximum time to wait for subprocess to terminate (seconds)
  1565. Raises:
  1566. GitProtocolError: If subprocess doesn't terminate within timeout
  1567. """
  1568. self.proc.stdin.close()
  1569. self.proc.stdout.close()
  1570. if self.proc.stderr:
  1571. self.proc.stderr.close()
  1572. try:
  1573. self.proc.wait(timeout=timeout)
  1574. except subprocess.TimeoutExpired as e:
  1575. self.proc.kill()
  1576. self.proc.wait()
  1577. raise GitProtocolError(
  1578. f"Git subprocess did not terminate within {timeout} seconds; killed it."
  1579. ) from e
  1580. def find_git_command() -> list[str]:
  1581. """Find command to run for system Git (usually C Git)."""
  1582. if sys.platform == "win32": # support .exe, .bat and .cmd
  1583. try: # to avoid overhead
  1584. import pywintypes
  1585. import win32api
  1586. except ImportError: # run through cmd.exe with some overhead
  1587. return ["cmd", "/c", "git"]
  1588. else:
  1589. try:
  1590. status, git = win32api.FindExecutable("git")
  1591. return [git]
  1592. except pywintypes.error:
  1593. return ["cmd", "/c", "git"]
  1594. else:
  1595. return ["git"]
  1596. class SubprocessGitClient(TraditionalGitClient):
  1597. """Git client that talks to a server using a subprocess."""
  1598. @classmethod
  1599. def from_parsedurl(cls, parsedurl, **kwargs):
  1600. """Create an instance of SubprocessGitClient from a parsed URL.
  1601. Args:
  1602. parsedurl: Result of urlparse()
  1603. **kwargs: Additional arguments for the client
  1604. Returns:
  1605. A SubprocessGitClient instance
  1606. """
  1607. return cls(**kwargs)
  1608. git_command: Optional[str] = None
  1609. def _connect(
  1610. self,
  1611. service: bytes,
  1612. path: Union[bytes, str],
  1613. protocol_version: Optional[int] = None,
  1614. ) -> tuple[Protocol, Callable[[], bool], Optional[IO[bytes]]]:
  1615. if not isinstance(service, bytes):
  1616. raise TypeError(service)
  1617. if isinstance(path, bytes):
  1618. path = path.decode(self._remote_path_encoding)
  1619. if self.git_command is None:
  1620. git_command = find_git_command()
  1621. argv = [*git_command, service.decode("ascii"), path]
  1622. p = subprocess.Popen(
  1623. argv,
  1624. bufsize=0,
  1625. stdin=subprocess.PIPE,
  1626. stdout=subprocess.PIPE,
  1627. stderr=subprocess.PIPE,
  1628. )
  1629. pw = SubprocessWrapper(p)
  1630. return (
  1631. Protocol(
  1632. pw.read,
  1633. pw.write,
  1634. pw.close,
  1635. report_activity=self._report_activity,
  1636. ),
  1637. pw.can_read,
  1638. p.stderr,
  1639. )
  1640. class LocalGitClient(GitClient):
  1641. """Git Client that just uses a local on-disk repository."""
  1642. def __init__(
  1643. self,
  1644. thin_packs: bool = True,
  1645. report_activity=None,
  1646. config: Optional[Config] = None,
  1647. ) -> None:
  1648. """Create a new LocalGitClient instance.
  1649. Args:
  1650. thin_packs: Whether or not thin packs should be retrieved
  1651. report_activity: Optional callback for reporting transport
  1652. activity.
  1653. """
  1654. self._report_activity = report_activity
  1655. # Ignore the thin_packs argument
  1656. def get_url(self, path):
  1657. """Get the URL for a local file path.
  1658. Args:
  1659. path: Local file path
  1660. Returns:
  1661. file:// URL for the path
  1662. """
  1663. return urlunsplit(("file", "", path, "", ""))
  1664. @classmethod
  1665. def from_parsedurl(cls, parsedurl, **kwargs):
  1666. """Create an instance of LocalGitClient from a parsed URL.
  1667. Args:
  1668. parsedurl: Result of urlparse()
  1669. **kwargs: Additional arguments for the client
  1670. Returns:
  1671. A LocalGitClient instance
  1672. """
  1673. return cls(**kwargs)
  1674. @classmethod
  1675. def _open_repo(cls, path):
  1676. """Open a local repository.
  1677. Args:
  1678. path: Repository path (as bytes or str)
  1679. Returns:
  1680. Repo instance wrapped in a closing context manager
  1681. """
  1682. if not isinstance(path, str):
  1683. path = os.fsdecode(path)
  1684. return closing(Repo(path))
  1685. def send_pack(self, path, update_refs, generate_pack_data, progress=None):
  1686. """Upload a pack to a local on-disk repository.
  1687. Args:
  1688. path: Repository path (as bytestring)
  1689. update_refs: Function to determine changes to remote refs.
  1690. Receive dict with existing remote refs, returns dict with
  1691. changed refs (name -> sha, where sha=ZERO_SHA for deletions)
  1692. with number of items and pack data to upload.
  1693. progress: Optional progress function
  1694. Returns:
  1695. SendPackResult
  1696. Raises:
  1697. SendPackError: if server rejects the pack data
  1698. """
  1699. if not progress:
  1700. def progress(x) -> None:
  1701. pass
  1702. with self._open_repo(path) as target:
  1703. old_refs = target.get_refs()
  1704. new_refs = update_refs(dict(old_refs))
  1705. have = [sha1 for sha1 in old_refs.values() if sha1 != ZERO_SHA]
  1706. want = []
  1707. for refname, new_sha1 in new_refs.items():
  1708. if (
  1709. new_sha1 not in have
  1710. and new_sha1 not in want
  1711. and new_sha1 != ZERO_SHA
  1712. ):
  1713. want.append(new_sha1)
  1714. if not want and set(new_refs.items()).issubset(set(old_refs.items())):
  1715. return SendPackResult(new_refs, ref_status={})
  1716. target.object_store.add_pack_data(
  1717. *generate_pack_data(have, want, ofs_delta=True)
  1718. )
  1719. ref_status = {}
  1720. for refname, new_sha1 in new_refs.items():
  1721. old_sha1 = old_refs.get(refname, ZERO_SHA)
  1722. if new_sha1 != ZERO_SHA:
  1723. if not target.refs.set_if_equals(refname, old_sha1, new_sha1):
  1724. msg = f"unable to set {refname} to {new_sha1}"
  1725. progress(msg)
  1726. ref_status[refname] = msg
  1727. else:
  1728. if not target.refs.remove_if_equals(refname, old_sha1):
  1729. progress(f"unable to remove {refname}")
  1730. ref_status[refname] = "unable to remove"
  1731. return SendPackResult(new_refs, ref_status=ref_status)
  1732. def fetch(
  1733. self,
  1734. path: str,
  1735. target: BaseRepo,
  1736. determine_wants: Optional[
  1737. Callable[[dict[bytes, bytes], Optional[int]], list[bytes]]
  1738. ] = None,
  1739. progress: Optional[Callable[[bytes], None]] = None,
  1740. depth: Optional[int] = None,
  1741. ref_prefix: Optional[list[Ref]] = None,
  1742. filter_spec: Optional[bytes] = None,
  1743. protocol_version: Optional[int] = None,
  1744. **kwargs,
  1745. ):
  1746. """Fetch into a target repository.
  1747. Args:
  1748. path: Path to fetch from (as bytestring)
  1749. target: Target repository to fetch into
  1750. determine_wants: Optional function determine what refs
  1751. to fetch. Receives dictionary of name->sha, should return
  1752. list of shas to fetch. Defaults to all shas.
  1753. progress: Optional progress function
  1754. depth: Shallow fetch depth
  1755. ref_prefix: List of prefixes of desired references, as a list of
  1756. bytestrings. Filtering is done by the server if supported, and
  1757. client side otherwise.
  1758. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  1759. Only used if the server supports the Git protocol-v2 'filter'
  1760. feature, and ignored otherwise.
  1761. Returns:
  1762. FetchPackResult object
  1763. """
  1764. with self._open_repo(path) as r:
  1765. refs = r.fetch(
  1766. target,
  1767. determine_wants=determine_wants,
  1768. progress=progress,
  1769. depth=depth,
  1770. )
  1771. return FetchPackResult(refs, r.refs.get_symrefs(), agent_string())
  1772. def fetch_pack(
  1773. self,
  1774. path,
  1775. determine_wants,
  1776. graph_walker,
  1777. pack_data,
  1778. progress=None,
  1779. depth: Optional[int] = None,
  1780. ref_prefix: Optional[list[Ref]] = None,
  1781. filter_spec: Optional[bytes] = None,
  1782. protocol_version: Optional[int] = None,
  1783. ) -> FetchPackResult:
  1784. """Retrieve a pack from a local on-disk repository.
  1785. Args:
  1786. path: Remote path to fetch from
  1787. determine_wants: Function determine what refs
  1788. to fetch. Receives dictionary of name->sha, should return
  1789. list of shas to fetch.
  1790. graph_walker: Object with next() and ack().
  1791. pack_data: Callback called for each bit of data in the pack
  1792. progress: Callback for progress reports (strings)
  1793. depth: Shallow fetch depth
  1794. ref_prefix: List of prefixes of desired references, as a list of
  1795. bytestrings. Filtering is done by the server if supported, and
  1796. client side otherwise.
  1797. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  1798. Only used if the server supports the Git protocol-v2 'filter'
  1799. feature, and ignored otherwise.
  1800. Returns:
  1801. FetchPackResult object
  1802. """
  1803. with self._open_repo(path) as r:
  1804. missing_objects = r.find_missing_objects(
  1805. determine_wants, graph_walker, progress=progress, depth=depth
  1806. )
  1807. other_haves = missing_objects.get_remote_has()
  1808. object_ids = list(missing_objects)
  1809. symrefs = r.refs.get_symrefs()
  1810. agent = agent_string()
  1811. # Did the process short-circuit (e.g. in a stateless RPC call)?
  1812. # Note that the client still expects a 0-object pack in most cases.
  1813. if object_ids is None:
  1814. return FetchPackResult(None, symrefs, agent)
  1815. write_pack_from_container(
  1816. pack_data, r.object_store, object_ids, other_haves=other_haves
  1817. )
  1818. return FetchPackResult(r.get_refs(), symrefs, agent)
  1819. def get_refs(
  1820. self,
  1821. path,
  1822. protocol_version: Optional[int] = None,
  1823. ref_prefix: Optional[list[Ref]] = None,
  1824. ):
  1825. """Retrieve the current refs from a local on-disk repository."""
  1826. with self._open_repo(path) as target:
  1827. refs = target.get_refs()
  1828. # Extract symrefs from the local repository
  1829. symrefs = {}
  1830. for ref in refs:
  1831. try:
  1832. # Check if this ref is symbolic by reading it directly
  1833. ref_value = target.refs.read_ref(ref)
  1834. if ref_value and ref_value.startswith(SYMREF):
  1835. # Extract the target from the symref
  1836. symrefs[ref] = ref_value[len(SYMREF) :]
  1837. except (KeyError, ValueError):
  1838. # Not a symbolic ref or error reading it
  1839. pass
  1840. return LsRemoteResult(refs, symrefs)
  1841. class BundleClient(GitClient):
  1842. """Git Client that reads from a bundle file."""
  1843. def __init__(
  1844. self,
  1845. thin_packs: bool = True,
  1846. report_activity=None,
  1847. config: Optional[Config] = None,
  1848. ) -> None:
  1849. """Create a new BundleClient instance.
  1850. Args:
  1851. thin_packs: Whether or not thin packs should be retrieved
  1852. report_activity: Optional callback for reporting transport
  1853. activity.
  1854. """
  1855. self._report_activity = report_activity
  1856. def get_url(self, path):
  1857. """Get the URL for a bundle file path.
  1858. Args:
  1859. path: Bundle file path
  1860. Returns:
  1861. The path unchanged (bundle files use local paths)
  1862. """
  1863. return path
  1864. @classmethod
  1865. def from_parsedurl(cls, parsedurl, **kwargs):
  1866. """Create an instance of BundleClient from a parsed URL.
  1867. Args:
  1868. parsedurl: Result of urlparse()
  1869. **kwargs: Additional arguments for the client
  1870. Returns:
  1871. A BundleClient instance
  1872. """
  1873. return cls(**kwargs)
  1874. @classmethod
  1875. def _is_bundle_file(cls, path):
  1876. """Check if a file is a git bundle by reading the first line."""
  1877. try:
  1878. with open(path, "rb") as f:
  1879. first_line = f.readline()
  1880. return first_line in (b"# v2 git bundle\n", b"# v3 git bundle\n")
  1881. except OSError:
  1882. return False
  1883. @classmethod
  1884. def _open_bundle(cls, path):
  1885. """Open and parse a bundle file.
  1886. Args:
  1887. path: Path to the bundle file (bytes or str)
  1888. Returns:
  1889. Bundle object with parsed metadata
  1890. Raises:
  1891. AssertionError: If bundle format is unsupported
  1892. """
  1893. if not isinstance(path, str):
  1894. path = os.fsdecode(path)
  1895. # Read bundle metadata without PackData to avoid file handle issues
  1896. with open(path, "rb") as f:
  1897. from dulwich.bundle import Bundle
  1898. version = None
  1899. firstline = f.readline()
  1900. if firstline == b"# v2 git bundle\n":
  1901. version = 2
  1902. elif firstline == b"# v3 git bundle\n":
  1903. version = 3
  1904. else:
  1905. raise AssertionError(f"unsupported bundle format header: {firstline!r}")
  1906. capabilities = {}
  1907. prerequisites = []
  1908. references = {}
  1909. line = f.readline()
  1910. if version >= 3:
  1911. while line.startswith(b"@"):
  1912. line = line[1:].rstrip(b"\n")
  1913. try:
  1914. key, value_bytes = line.split(b"=", 1)
  1915. value = value_bytes.decode("utf-8")
  1916. except ValueError:
  1917. key = line
  1918. value = None
  1919. capabilities[key.decode("utf-8")] = value
  1920. line = f.readline()
  1921. while line.startswith(b"-"):
  1922. (obj_id, comment) = line[1:].rstrip(b"\n").split(b" ", 1)
  1923. prerequisites.append((obj_id, comment.decode("utf-8")))
  1924. line = f.readline()
  1925. while line != b"\n":
  1926. (obj_id, ref) = line.rstrip(b"\n").split(b" ", 1)
  1927. references[ref] = obj_id
  1928. line = f.readline()
  1929. # Don't read PackData here, we'll do it later
  1930. bundle = Bundle()
  1931. bundle.version = version
  1932. bundle.capabilities = capabilities
  1933. bundle.prerequisites = prerequisites
  1934. bundle.references = references
  1935. bundle.pack_data = None # Will be read on demand
  1936. return bundle
  1937. @staticmethod
  1938. def _skip_to_pack_data(f, version):
  1939. """Skip to the pack data section in a bundle file.
  1940. Args:
  1941. f: File object positioned at the beginning of the bundle
  1942. version: Bundle format version (2 or 3)
  1943. Raises:
  1944. AssertionError: If bundle header is invalid
  1945. """
  1946. # Skip header
  1947. header = f.readline()
  1948. if header not in (b"# v2 git bundle\n", b"# v3 git bundle\n"):
  1949. raise AssertionError(f"Invalid bundle header: {header!r}")
  1950. line = f.readline()
  1951. # Skip capabilities (v3 only)
  1952. if version >= 3:
  1953. while line.startswith(b"@"):
  1954. line = f.readline()
  1955. # Skip prerequisites
  1956. while line.startswith(b"-"):
  1957. line = f.readline()
  1958. # Skip references
  1959. while line != b"\n":
  1960. line = f.readline()
  1961. # Now at pack data
  1962. def send_pack(self, path, update_refs, generate_pack_data, progress=None):
  1963. """Upload is not supported for bundle files."""
  1964. raise NotImplementedError("Bundle files are read-only")
  1965. def fetch(
  1966. self,
  1967. path: str,
  1968. target: BaseRepo,
  1969. determine_wants: Optional[
  1970. Callable[[dict[bytes, bytes], Optional[int]], list[bytes]]
  1971. ] = None,
  1972. progress: Optional[Callable[[bytes], None]] = None,
  1973. depth: Optional[int] = None,
  1974. ref_prefix: Optional[list[Ref]] = None,
  1975. filter_spec: Optional[bytes] = None,
  1976. protocol_version: Optional[int] = None,
  1977. **kwargs,
  1978. ):
  1979. """Fetch into a target repository from a bundle file."""
  1980. bundle = self._open_bundle(path)
  1981. # Get references from bundle
  1982. refs = dict(bundle.references)
  1983. # Determine what we want to fetch
  1984. if determine_wants is None:
  1985. _ = list(refs.values())
  1986. else:
  1987. _ = determine_wants(refs, None)
  1988. # Add pack data to target repository
  1989. # Need to reopen the file for pack data access
  1990. with open(path, "rb") as pack_file:
  1991. # Skip to pack data section
  1992. BundleClient._skip_to_pack_data(pack_file, bundle.version)
  1993. # Read pack data into memory to avoid file positioning issues
  1994. pack_bytes = pack_file.read()
  1995. # Create PackData from in-memory bytes
  1996. from io import BytesIO
  1997. pack_io = BytesIO(pack_bytes)
  1998. pack_data = PackData.from_file(pack_io)
  1999. target.object_store.add_pack_data(len(pack_data), pack_data.iter_unpacked())
  2000. # Apply ref filtering if specified
  2001. if ref_prefix:
  2002. filtered_refs = {}
  2003. for ref_name, ref_value in refs.items():
  2004. for prefix in ref_prefix:
  2005. if ref_name.startswith(prefix):
  2006. filtered_refs[ref_name] = ref_value
  2007. break
  2008. refs = filtered_refs
  2009. return FetchPackResult(refs, {}, agent_string())
  2010. def fetch_pack(
  2011. self,
  2012. path,
  2013. determine_wants,
  2014. graph_walker,
  2015. pack_data,
  2016. progress=None,
  2017. depth: Optional[int] = None,
  2018. ref_prefix: Optional[list[Ref]] = None,
  2019. filter_spec: Optional[bytes] = None,
  2020. protocol_version: Optional[int] = None,
  2021. ) -> FetchPackResult:
  2022. """Retrieve a pack from a bundle file."""
  2023. bundle = self._open_bundle(path)
  2024. # Get references from bundle
  2025. refs = dict(bundle.references)
  2026. # Determine what we want to fetch
  2027. _ = determine_wants(refs)
  2028. # Write pack data to the callback
  2029. # Need to reopen the file for pack data access
  2030. with open(path, "rb") as pack_file:
  2031. # Skip to pack data section
  2032. BundleClient._skip_to_pack_data(pack_file, bundle.version)
  2033. # Read pack data and write it to the callback
  2034. pack_bytes = pack_file.read()
  2035. pack_data(pack_bytes)
  2036. # Apply ref filtering if specified
  2037. if ref_prefix:
  2038. filtered_refs = {}
  2039. for ref_name, ref_value in refs.items():
  2040. for prefix in ref_prefix:
  2041. if ref_name.startswith(prefix):
  2042. filtered_refs[ref_name] = ref_value
  2043. break
  2044. refs = filtered_refs
  2045. return FetchPackResult(refs, {}, agent_string())
  2046. def get_refs(
  2047. self,
  2048. path,
  2049. protocol_version: Optional[int] = None,
  2050. ref_prefix: Optional[list[Ref]] = None,
  2051. ):
  2052. """Retrieve the current refs from a bundle file."""
  2053. bundle = self._open_bundle(path)
  2054. refs = dict(bundle.references)
  2055. # Apply ref filtering if specified
  2056. if ref_prefix:
  2057. filtered_refs = {}
  2058. for ref_name, ref_value in refs.items():
  2059. for prefix in ref_prefix:
  2060. if ref_name.startswith(prefix):
  2061. filtered_refs[ref_name] = ref_value
  2062. break
  2063. refs = filtered_refs
  2064. return LsRemoteResult(refs, {})
  2065. # What Git client to use for local access
  2066. default_local_git_client_cls = LocalGitClient
  2067. class SSHVendor:
  2068. """A client side SSH implementation."""
  2069. def run_command(
  2070. self,
  2071. host,
  2072. command,
  2073. username=None,
  2074. port=None,
  2075. password=None,
  2076. key_filename=None,
  2077. ssh_command=None,
  2078. protocol_version: Optional[int] = None,
  2079. ):
  2080. """Connect to an SSH server.
  2081. Run a command remotely and return a file-like object for interaction
  2082. with the remote command.
  2083. Args:
  2084. host: Host name
  2085. command: Command to run (as argv array)
  2086. username: Optional ame of user to log in as
  2087. port: Optional SSH port to use
  2088. password: Optional ssh password for login or private key
  2089. key_filename: Optional path to private keyfile
  2090. ssh_command: Optional SSH command
  2091. protocol_version: Desired Git protocol version. By default the highest
  2092. mutually supported protocol version will be used.
  2093. """
  2094. raise NotImplementedError(self.run_command)
  2095. class StrangeHostname(Exception):
  2096. """Refusing to connect to strange SSH hostname."""
  2097. def __init__(self, hostname) -> None:
  2098. super().__init__(hostname)
  2099. class SubprocessSSHVendor(SSHVendor):
  2100. """SSH vendor that shells out to the local 'ssh' command."""
  2101. def run_command(
  2102. self,
  2103. host,
  2104. command,
  2105. username=None,
  2106. port=None,
  2107. password=None,
  2108. key_filename=None,
  2109. ssh_command=None,
  2110. protocol_version: Optional[int] = None,
  2111. ):
  2112. if password is not None:
  2113. raise NotImplementedError(
  2114. "Setting password not supported by SubprocessSSHVendor."
  2115. )
  2116. if ssh_command:
  2117. import shlex
  2118. args = [*shlex.split(ssh_command, posix=sys.platform != "win32"), "-x"]
  2119. else:
  2120. args = ["ssh", "-x"]
  2121. if port:
  2122. args.extend(["-p", str(port)])
  2123. if key_filename:
  2124. args.extend(["-i", str(key_filename)])
  2125. if protocol_version is None:
  2126. protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_FETCH
  2127. if protocol_version > 0:
  2128. args.extend(["-o", f"SetEnv GIT_PROTOCOL=version={protocol_version}"])
  2129. if username:
  2130. host = f"{username}@{host}"
  2131. if host.startswith("-"):
  2132. raise StrangeHostname(hostname=host)
  2133. args.append(host)
  2134. proc = subprocess.Popen(
  2135. [*args, command],
  2136. bufsize=0,
  2137. stdin=subprocess.PIPE,
  2138. stdout=subprocess.PIPE,
  2139. stderr=subprocess.PIPE,
  2140. )
  2141. return SubprocessWrapper(proc)
  2142. class PLinkSSHVendor(SSHVendor):
  2143. """SSH vendor that shells out to the local 'plink' command."""
  2144. def run_command(
  2145. self,
  2146. host,
  2147. command,
  2148. username=None,
  2149. port=None,
  2150. password=None,
  2151. key_filename=None,
  2152. ssh_command=None,
  2153. protocol_version: Optional[int] = None,
  2154. ):
  2155. if ssh_command:
  2156. import shlex
  2157. args = [*shlex.split(ssh_command, posix=sys.platform != "win32"), "-ssh"]
  2158. elif sys.platform == "win32":
  2159. args = ["plink.exe", "-ssh"]
  2160. else:
  2161. args = ["plink", "-ssh"]
  2162. if password is not None:
  2163. import warnings
  2164. warnings.warn(
  2165. "Invoking PLink with a password exposes the password in the "
  2166. "process list."
  2167. )
  2168. args.extend(["-pw", str(password)])
  2169. if port:
  2170. args.extend(["-P", str(port)])
  2171. if key_filename:
  2172. args.extend(["-i", str(key_filename)])
  2173. if username:
  2174. host = f"{username}@{host}"
  2175. if host.startswith("-"):
  2176. raise StrangeHostname(hostname=host)
  2177. args.append(host)
  2178. # plink.exe does not provide a way to pass environment variables
  2179. # via the command line. The best we can do is set an environment
  2180. # variable and hope that plink will pass it to the server. If this
  2181. # does not work then the server should behave as if we had requested
  2182. # protocol version 0.
  2183. env = copy.deepcopy(os.environ)
  2184. if protocol_version is None:
  2185. protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_FETCH
  2186. if protocol_version > 0:
  2187. env["GIT_PROTOCOL"] = f"version={protocol_version}"
  2188. proc = subprocess.Popen(
  2189. [*args, command],
  2190. bufsize=0,
  2191. stdin=subprocess.PIPE,
  2192. stdout=subprocess.PIPE,
  2193. stderr=subprocess.PIPE,
  2194. env=env,
  2195. )
  2196. return SubprocessWrapper(proc)
  2197. def ParamikoSSHVendor(**kwargs):
  2198. import warnings
  2199. warnings.warn(
  2200. "ParamikoSSHVendor has been moved to dulwich.contrib.paramiko_vendor.",
  2201. DeprecationWarning,
  2202. )
  2203. from .contrib.paramiko_vendor import ParamikoSSHVendor
  2204. return ParamikoSSHVendor(**kwargs)
  2205. # Can be overridden by users
  2206. get_ssh_vendor: Callable[[], SSHVendor] = SubprocessSSHVendor
  2207. class SSHGitClient(TraditionalGitClient):
  2208. def __init__(
  2209. self,
  2210. host,
  2211. port=None,
  2212. username=None,
  2213. vendor=None,
  2214. config=None,
  2215. password=None,
  2216. key_filename=None,
  2217. ssh_command=None,
  2218. **kwargs,
  2219. ) -> None:
  2220. self.host = host
  2221. self.port = port
  2222. self.username = username
  2223. self.password = password
  2224. self.key_filename = key_filename
  2225. # Priority: ssh_command parameter, then env vars, then core.sshCommand config
  2226. if ssh_command:
  2227. self.ssh_command = ssh_command
  2228. else:
  2229. # Check environment variables first
  2230. self.ssh_command = os.environ.get(
  2231. "GIT_SSH_COMMAND", os.environ.get("GIT_SSH")
  2232. )
  2233. # Fall back to config if no environment variable set
  2234. if not self.ssh_command and config is not None:
  2235. try:
  2236. config_ssh_command = config.get((b"core",), b"sshCommand")
  2237. self.ssh_command = (
  2238. config_ssh_command.decode() if config_ssh_command else None
  2239. )
  2240. except KeyError:
  2241. pass
  2242. super().__init__(**kwargs)
  2243. self.alternative_paths: dict[bytes, bytes] = {}
  2244. if vendor is not None:
  2245. self.ssh_vendor = vendor
  2246. else:
  2247. self.ssh_vendor = get_ssh_vendor()
  2248. def get_url(self, path):
  2249. netloc = self.host
  2250. if self.port is not None:
  2251. netloc += f":{self.port}"
  2252. if self.username is not None:
  2253. netloc = urlquote(self.username, "@/:") + "@" + netloc
  2254. return urlunsplit(("ssh", netloc, path, "", ""))
  2255. @classmethod
  2256. def from_parsedurl(cls, parsedurl, **kwargs):
  2257. return cls(
  2258. host=parsedurl.hostname,
  2259. port=parsedurl.port,
  2260. username=parsedurl.username,
  2261. **kwargs,
  2262. )
  2263. def _get_cmd_path(self, cmd):
  2264. cmd = self.alternative_paths.get(cmd, b"git-" + cmd)
  2265. assert isinstance(cmd, bytes)
  2266. return cmd
  2267. def _connect(
  2268. self,
  2269. cmd: bytes,
  2270. path: Union[str, bytes],
  2271. protocol_version: Optional[int] = None,
  2272. ) -> tuple[Protocol, Callable[[], bool], Optional[IO[bytes]]]:
  2273. if not isinstance(cmd, bytes):
  2274. raise TypeError(cmd)
  2275. if isinstance(path, bytes):
  2276. path = path.decode(self._remote_path_encoding)
  2277. if path.startswith("/~"):
  2278. path = path[1:]
  2279. argv = (
  2280. self._get_cmd_path(cmd).decode(self._remote_path_encoding)
  2281. + " '"
  2282. + path
  2283. + "'"
  2284. )
  2285. kwargs = {}
  2286. if self.password is not None:
  2287. kwargs["password"] = self.password
  2288. if self.key_filename is not None:
  2289. kwargs["key_filename"] = self.key_filename
  2290. # GIT_SSH_COMMAND takes precedence over GIT_SSH
  2291. if self.ssh_command is not None:
  2292. kwargs["ssh_command"] = self.ssh_command
  2293. con = self.ssh_vendor.run_command(
  2294. self.host,
  2295. argv,
  2296. port=self.port,
  2297. username=self.username,
  2298. protocol_version=protocol_version,
  2299. **kwargs,
  2300. )
  2301. return (
  2302. Protocol(
  2303. con.read,
  2304. con.write,
  2305. con.close,
  2306. report_activity=self._report_activity,
  2307. ),
  2308. con.can_read,
  2309. getattr(con, "stderr", None),
  2310. )
  2311. def default_user_agent_string():
  2312. # Start user agent with "git/", because GitHub requires this. :-( See
  2313. # https://github.com/jelmer/dulwich/issues/562 for details.
  2314. return "git/dulwich/{}".format(".".join([str(x) for x in dulwich.__version__]))
  2315. def default_urllib3_manager(
  2316. config,
  2317. pool_manager_cls=None,
  2318. proxy_manager_cls=None,
  2319. base_url=None,
  2320. timeout=None,
  2321. **override_kwargs,
  2322. ) -> Union["urllib3.ProxyManager", "urllib3.PoolManager"]:
  2323. """Return urllib3 connection pool manager.
  2324. Honour detected proxy configurations.
  2325. Args:
  2326. config: `dulwich.config.ConfigDict` instance with Git configuration.
  2327. timeout: Timeout for HTTP requests in seconds
  2328. override_kwargs: Additional arguments for `urllib3.ProxyManager`
  2329. Returns:
  2330. Either pool_manager_cls (defaults to `urllib3.ProxyManager`) instance for
  2331. proxy configurations, proxy_manager_cls
  2332. (defaults to `urllib3.PoolManager`) instance otherwise
  2333. """
  2334. proxy_server = user_agent = None
  2335. ca_certs = ssl_verify = None
  2336. if proxy_server is None:
  2337. for proxyname in ("https_proxy", "http_proxy", "all_proxy"):
  2338. proxy_server = os.environ.get(proxyname)
  2339. if proxy_server:
  2340. break
  2341. if proxy_server:
  2342. if check_for_proxy_bypass(base_url):
  2343. proxy_server = None
  2344. if config is not None:
  2345. if proxy_server is None:
  2346. try:
  2347. proxy_server = config.get(b"http", b"proxy")
  2348. except KeyError:
  2349. pass
  2350. try:
  2351. user_agent = config.get(b"http", b"useragent")
  2352. except KeyError:
  2353. pass
  2354. # TODO(jelmer): Support per-host settings
  2355. try:
  2356. ssl_verify = config.get_boolean(b"http", b"sslVerify")
  2357. except KeyError:
  2358. ssl_verify = True
  2359. try:
  2360. ca_certs = config.get(b"http", b"sslCAInfo")
  2361. except KeyError:
  2362. ca_certs = None
  2363. # Check for timeout configuration
  2364. if timeout is None:
  2365. try:
  2366. timeout = config.get(b"http", b"timeout")
  2367. if timeout is not None:
  2368. timeout = int(timeout)
  2369. except KeyError:
  2370. pass
  2371. if user_agent is None:
  2372. user_agent = default_user_agent_string()
  2373. headers = {"User-agent": user_agent}
  2374. kwargs = {
  2375. "ca_certs": ca_certs,
  2376. }
  2377. # Add timeout if specified
  2378. if timeout is not None:
  2379. kwargs["timeout"] = timeout
  2380. if ssl_verify is True:
  2381. kwargs["cert_reqs"] = "CERT_REQUIRED"
  2382. elif ssl_verify is False:
  2383. kwargs["cert_reqs"] = "CERT_NONE"
  2384. else:
  2385. # Default to SSL verification
  2386. kwargs["cert_reqs"] = "CERT_REQUIRED"
  2387. kwargs.update(override_kwargs)
  2388. import urllib3
  2389. if proxy_server is not None:
  2390. if proxy_manager_cls is None:
  2391. proxy_manager_cls = urllib3.ProxyManager
  2392. if not isinstance(proxy_server, str):
  2393. proxy_server = proxy_server.decode()
  2394. proxy_server_url = urlparse(proxy_server)
  2395. if proxy_server_url.username is not None:
  2396. proxy_headers = urllib3.make_headers(
  2397. proxy_basic_auth=f"{proxy_server_url.username}:{proxy_server_url.password or ''}" # type: ignore
  2398. )
  2399. else:
  2400. proxy_headers = {}
  2401. manager = proxy_manager_cls(
  2402. proxy_server, proxy_headers=proxy_headers, headers=headers, **kwargs
  2403. )
  2404. else:
  2405. if pool_manager_cls is None:
  2406. pool_manager_cls = urllib3.PoolManager
  2407. manager = pool_manager_cls(headers=headers, **kwargs)
  2408. return manager
  2409. def check_for_proxy_bypass(base_url) -> bool:
  2410. # Check if a proxy bypass is defined with the no_proxy environment variable
  2411. if base_url: # only check if base_url is provided
  2412. no_proxy_str = os.environ.get("no_proxy")
  2413. if no_proxy_str:
  2414. # implementation based on curl behavior: https://curl.se/libcurl/c/CURLOPT_NOPROXY.html
  2415. # get hostname of provided parsed url
  2416. parsed_url = urlparse(base_url)
  2417. hostname = parsed_url.hostname
  2418. if hostname:
  2419. import ipaddress
  2420. # check if hostname is an ip address
  2421. try:
  2422. hostname_ip = ipaddress.ip_address(hostname)
  2423. except ValueError:
  2424. hostname_ip = None
  2425. no_proxy_values = no_proxy_str.split(",")
  2426. for no_proxy_value in no_proxy_values:
  2427. no_proxy_value = no_proxy_value.strip()
  2428. if no_proxy_value:
  2429. no_proxy_value = no_proxy_value.lower()
  2430. no_proxy_value = no_proxy_value.lstrip(
  2431. "."
  2432. ) # ignore leading dots
  2433. if hostname_ip:
  2434. # check if no_proxy_value is a ip network
  2435. try:
  2436. no_proxy_value_network = ipaddress.ip_network(
  2437. no_proxy_value, strict=False
  2438. )
  2439. except ValueError:
  2440. no_proxy_value_network = None
  2441. if no_proxy_value_network:
  2442. # if hostname is a ip address and no_proxy_value is a ip network -> check if ip address is part of network
  2443. if hostname_ip in no_proxy_value_network:
  2444. return True
  2445. if no_proxy_value == "*":
  2446. # '*' is special case for always bypass proxy
  2447. return True
  2448. if hostname == no_proxy_value:
  2449. return True
  2450. no_proxy_value = (
  2451. "." + no_proxy_value
  2452. ) # add a dot to only match complete domains
  2453. if hostname.endswith(no_proxy_value):
  2454. return True
  2455. return False
  2456. class AbstractHttpGitClient(GitClient):
  2457. """Abstract base class for HTTP Git Clients.
  2458. This is agonistic of the actual HTTP implementation.
  2459. Subclasses should provide an implementation of the
  2460. _http_request method.
  2461. """
  2462. def __init__(self, base_url, dumb=False, **kwargs) -> None:
  2463. self._base_url = base_url.rstrip("/") + "/"
  2464. self.dumb = dumb
  2465. GitClient.__init__(self, **kwargs)
  2466. def _http_request(self, url, headers=None, data=None, raise_for_status=True):
  2467. """Perform HTTP request.
  2468. Args:
  2469. url: Request URL.
  2470. headers: Optional custom headers to override defaults.
  2471. data: Request data.
  2472. Returns:
  2473. Tuple (response, read), where response is an urllib3
  2474. response object with additional content_type and
  2475. redirect_location properties, and read is a consumable read
  2476. method for the response data.
  2477. Raises:
  2478. GitProtocolError
  2479. """
  2480. raise NotImplementedError(self._http_request)
  2481. def _discover_references(
  2482. self,
  2483. service,
  2484. base_url,
  2485. protocol_version: Optional[int] = None,
  2486. ref_prefix: Optional[list[Ref]] = None,
  2487. ) -> tuple[
  2488. dict[Ref, ObjectID], set[bytes], str, dict[Ref, Ref], dict[Ref, ObjectID]
  2489. ]:
  2490. if (
  2491. protocol_version is not None
  2492. and protocol_version not in GIT_PROTOCOL_VERSIONS
  2493. ):
  2494. raise ValueError(f"unknown Git protocol version {protocol_version}")
  2495. assert base_url[-1] == "/"
  2496. tail = "info/refs"
  2497. headers = {"Accept": "*/*"}
  2498. if self.dumb is not True:
  2499. tail += "?service={}".format(service.decode("ascii"))
  2500. # Enable protocol v2 only when fetching, not when pushing.
  2501. # Git does not yet implement push over protocol v2, and as of
  2502. # git version 2.37.3 git-http-backend's behaviour is erratic if
  2503. # we try: It responds with a Git-protocol-v1-style ref listing
  2504. # which lacks the "001f# service=git-receive-pack" marker.
  2505. if service == b"git-upload-pack":
  2506. if protocol_version is None:
  2507. self.protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_FETCH
  2508. else:
  2509. self.protocol_version = protocol_version
  2510. if self.protocol_version == 2:
  2511. headers["Git-Protocol"] = "version=2"
  2512. else:
  2513. self.protocol_version = DEFAULT_GIT_PROTOCOL_VERSION_SEND
  2514. url = urljoin(base_url, tail)
  2515. resp, read = self._http_request(url, headers)
  2516. if resp.redirect_location:
  2517. # Something changed (redirect!), so let's update the base URL
  2518. if not resp.redirect_location.endswith(tail):
  2519. raise GitProtocolError(
  2520. f"Redirected from URL {url} to URL {resp.redirect_location} without {tail}"
  2521. )
  2522. base_url = urljoin(url, resp.redirect_location[: -len(tail)])
  2523. try:
  2524. self.dumb = resp.content_type is None or not resp.content_type.startswith(
  2525. "application/x-git-"
  2526. )
  2527. if not self.dumb:
  2528. def begin_protocol_v2(proto):
  2529. nonlocal ref_prefix
  2530. server_capabilities = read_server_capabilities(proto.read_pkt_seq())
  2531. if ref_prefix is None:
  2532. ref_prefix = DEFAULT_REF_PREFIX
  2533. pkts = [
  2534. b"symrefs",
  2535. b"peel",
  2536. ]
  2537. for prefix in ref_prefix:
  2538. pkts.append(b"ref-prefix " + prefix)
  2539. body = b"".join(
  2540. [pkt_line(b"command=ls-refs\n"), b"0001", pkt_seq(*pkts)]
  2541. )
  2542. resp, read = self._smart_request(
  2543. service.decode("ascii"), base_url, body
  2544. )
  2545. proto = Protocol(read, None)
  2546. return server_capabilities, resp, read, proto
  2547. proto = Protocol(read, None) # type: ignore
  2548. server_protocol_version = negotiate_protocol_version(proto)
  2549. if server_protocol_version not in GIT_PROTOCOL_VERSIONS:
  2550. raise ValueError(
  2551. f"unknown Git protocol version {server_protocol_version} used by server"
  2552. )
  2553. if protocol_version and server_protocol_version > protocol_version:
  2554. raise ValueError(
  2555. f"bad Git protocol version {server_protocol_version} used by server"
  2556. )
  2557. self.protocol_version = server_protocol_version
  2558. if self.protocol_version == 2:
  2559. server_capabilities, resp, read, proto = begin_protocol_v2(proto)
  2560. (refs, symrefs, peeled) = read_pkt_refs_v2(proto.read_pkt_seq())
  2561. return refs, server_capabilities, base_url, symrefs, peeled
  2562. else:
  2563. try:
  2564. [pkt] = list(proto.read_pkt_seq())
  2565. except ValueError as exc:
  2566. raise GitProtocolError(
  2567. "unexpected number of packets received"
  2568. ) from exc
  2569. if pkt.rstrip(b"\n") != (b"# service=" + service):
  2570. raise GitProtocolError(
  2571. f"unexpected first line {pkt!r} from smart server"
  2572. )
  2573. # Github sends "version 2" after sending the service name.
  2574. # Try to negotiate protocol version 2 again.
  2575. server_protocol_version = negotiate_protocol_version(proto)
  2576. if server_protocol_version not in GIT_PROTOCOL_VERSIONS:
  2577. raise ValueError(
  2578. f"unknown Git protocol version {server_protocol_version} used by server"
  2579. )
  2580. if protocol_version and server_protocol_version > protocol_version:
  2581. raise ValueError(
  2582. f"bad Git protocol version {server_protocol_version} used by server"
  2583. )
  2584. self.protocol_version = server_protocol_version
  2585. if self.protocol_version == 2:
  2586. server_capabilities, resp, read, proto = begin_protocol_v2(
  2587. proto
  2588. )
  2589. (refs, symrefs, peeled) = read_pkt_refs_v2(proto.read_pkt_seq())
  2590. else:
  2591. (
  2592. refs,
  2593. server_capabilities,
  2594. ) = read_pkt_refs_v1(proto.read_pkt_seq())
  2595. (refs, peeled) = split_peeled_refs(refs)
  2596. (symrefs, agent) = _extract_symrefs_and_agent(
  2597. server_capabilities
  2598. )
  2599. if ref_prefix is not None:
  2600. refs = filter_ref_prefix(refs, ref_prefix)
  2601. return refs, server_capabilities, base_url, symrefs, peeled
  2602. else:
  2603. self.protocol_version = 0 # dumb servers only support protocol v0
  2604. # Read all the response data
  2605. data = b""
  2606. while True:
  2607. chunk = read(4096)
  2608. if not chunk:
  2609. break
  2610. data += chunk
  2611. (refs, peeled) = split_peeled_refs(read_info_refs(BytesIO(data)))
  2612. if ref_prefix is not None:
  2613. refs = filter_ref_prefix(refs, ref_prefix)
  2614. return refs, set(), base_url, {}, peeled
  2615. finally:
  2616. resp.close()
  2617. def _smart_request(self, service, url, data):
  2618. """Send a 'smart' HTTP request.
  2619. This is a simple wrapper around _http_request that sets
  2620. a couple of extra headers.
  2621. """
  2622. assert url[-1] == "/"
  2623. url = urljoin(url, service)
  2624. result_content_type = f"application/x-{service}-result"
  2625. headers = {
  2626. "Content-Type": f"application/x-{service}-request",
  2627. "Accept": result_content_type,
  2628. }
  2629. if self.protocol_version == 2:
  2630. headers["Git-Protocol"] = "version=2"
  2631. if isinstance(data, bytes):
  2632. headers["Content-Length"] = str(len(data))
  2633. resp, read = self._http_request(url, headers, data)
  2634. if resp.content_type.split(";")[0] != result_content_type:
  2635. raise GitProtocolError(
  2636. f"Invalid content-type from server: {resp.content_type}"
  2637. )
  2638. return resp, read
  2639. def send_pack(self, path, update_refs, generate_pack_data, progress=None):
  2640. """Upload a pack to a remote repository.
  2641. Args:
  2642. path: Repository path (as bytestring)
  2643. update_refs: Function to determine changes to remote refs.
  2644. Receives dict with existing remote refs, returns dict with
  2645. changed refs (name -> sha, where sha=ZERO_SHA for deletions)
  2646. generate_pack_data: Function that can return a tuple
  2647. with number of elements and pack data to upload.
  2648. progress: Optional progress function
  2649. Returns:
  2650. SendPackResult
  2651. Raises:
  2652. SendPackError: if server rejects the pack data
  2653. """
  2654. url = self._get_url(path)
  2655. old_refs, server_capabilities, url, symrefs, peeled = self._discover_references(
  2656. b"git-receive-pack", url
  2657. )
  2658. (
  2659. negotiated_capabilities,
  2660. agent,
  2661. ) = self._negotiate_receive_pack_capabilities(server_capabilities)
  2662. negotiated_capabilities.add(capability_agent())
  2663. if CAPABILITY_REPORT_STATUS in negotiated_capabilities:
  2664. self._report_status_parser = ReportStatusParser()
  2665. new_refs = update_refs(dict(old_refs))
  2666. if new_refs is None:
  2667. # Determine wants function is aborting the push.
  2668. return SendPackResult(old_refs, agent=agent, ref_status={})
  2669. if set(new_refs.items()).issubset(set(old_refs.items())):
  2670. return SendPackResult(new_refs, agent=agent, ref_status={})
  2671. if self.dumb:
  2672. raise NotImplementedError(self.fetch_pack)
  2673. def body_generator():
  2674. header_handler = _v1ReceivePackHeader(
  2675. negotiated_capabilities, old_refs, new_refs
  2676. )
  2677. for pkt in header_handler:
  2678. yield pkt_line(pkt)
  2679. pack_data_count, pack_data = generate_pack_data(
  2680. header_handler.have,
  2681. header_handler.want,
  2682. ofs_delta=(CAPABILITY_OFS_DELTA in negotiated_capabilities),
  2683. )
  2684. if self._should_send_pack(new_refs):
  2685. yield from PackChunkGenerator(pack_data_count, pack_data)
  2686. resp, read = self._smart_request("git-receive-pack", url, data=body_generator())
  2687. try:
  2688. resp_proto = Protocol(read, None)
  2689. ref_status = self._handle_receive_pack_tail(
  2690. resp_proto, negotiated_capabilities, progress
  2691. )
  2692. return SendPackResult(new_refs, agent=agent, ref_status=ref_status)
  2693. finally:
  2694. resp.close()
  2695. def fetch_pack(
  2696. self,
  2697. path,
  2698. determine_wants,
  2699. graph_walker,
  2700. pack_data,
  2701. progress=None,
  2702. depth: Optional[int] = None,
  2703. ref_prefix: Optional[list[Ref]] = None,
  2704. filter_spec=None,
  2705. protocol_version: Optional[int] = None,
  2706. ):
  2707. """Retrieve a pack from a git smart server.
  2708. Args:
  2709. path: Path to fetch from
  2710. determine_wants: Callback that returns list of commits to fetch
  2711. graph_walker: Object with next() and ack().
  2712. pack_data: Callback called for each bit of data in the pack
  2713. progress: Callback for progress reports (strings)
  2714. depth: Depth for request
  2715. ref_prefix: List of prefixes of desired references, as a list of
  2716. bytestrings. Filtering is done by the server if supported, and
  2717. client side otherwise.
  2718. filter_spec: A git-rev-list-style object filter spec, as bytestring.
  2719. Only used if the server supports the Git protocol-v2 'filter'
  2720. feature, and ignored otherwise.
  2721. protocol_version: Desired Git protocol version. By default the highest
  2722. mutually supported protocol version will be used.
  2723. Returns:
  2724. FetchPackResult object
  2725. """
  2726. url = self._get_url(path)
  2727. refs, server_capabilities, url, symrefs, peeled = self._discover_references(
  2728. b"git-upload-pack",
  2729. url,
  2730. protocol_version=protocol_version,
  2731. ref_prefix=ref_prefix,
  2732. )
  2733. (
  2734. negotiated_capabilities,
  2735. capa_symrefs,
  2736. agent,
  2737. ) = self._negotiate_upload_pack_capabilities(server_capabilities)
  2738. if not symrefs and capa_symrefs:
  2739. symrefs = capa_symrefs
  2740. if depth is not None:
  2741. wants = determine_wants(refs, depth=depth)
  2742. else:
  2743. wants = determine_wants(refs)
  2744. if wants is not None:
  2745. wants = [cid for cid in wants if cid != ZERO_SHA]
  2746. if not wants and not self.dumb:
  2747. return FetchPackResult(refs, symrefs, agent)
  2748. elif self.dumb:
  2749. # Use dumb HTTP protocol
  2750. from .dumb import DumbRemoteHTTPRepo
  2751. # Pass http_request function
  2752. dumb_repo = DumbRemoteHTTPRepo(
  2753. url, functools.partial(self._http_request, raise_for_status=False)
  2754. )
  2755. # Fetch pack data from dumb remote
  2756. pack_data_list = list(
  2757. dumb_repo.fetch_pack_data(
  2758. graph_walker, lambda refs: wants, progress=progress, depth=depth
  2759. )
  2760. )
  2761. symrefs[b"HEAD"] = dumb_repo.get_head()
  2762. # Write pack data
  2763. if pack_data:
  2764. from .pack import write_pack_data
  2765. # Write pack data directly using the unpacked objects
  2766. write_pack_data(
  2767. pack_data,
  2768. iter(pack_data_list),
  2769. num_records=len(pack_data_list),
  2770. progress=progress,
  2771. )
  2772. return FetchPackResult(refs, symrefs, agent)
  2773. req_data = BytesIO()
  2774. req_proto = Protocol(None, req_data.write) # type: ignore
  2775. (new_shallow, new_unshallow) = _handle_upload_pack_head(
  2776. req_proto,
  2777. negotiated_capabilities,
  2778. graph_walker,
  2779. wants,
  2780. can_read=None,
  2781. depth=depth,
  2782. protocol_version=self.protocol_version,
  2783. )
  2784. if self.protocol_version == 2:
  2785. data = pkt_line(b"command=fetch\n") + b"0001"
  2786. if CAPABILITY_THIN_PACK in self._fetch_capabilities:
  2787. data += pkt_line(b"thin-pack\n")
  2788. if (
  2789. find_capability(
  2790. negotiated_capabilities, CAPABILITY_FETCH, CAPABILITY_FILTER
  2791. )
  2792. and filter_spec
  2793. ):
  2794. data += pkt_line(b"filter %s\n" % filter_spec)
  2795. elif filter_spec:
  2796. self._warn_filter_objects()
  2797. data += req_data.getvalue()
  2798. else:
  2799. if filter_spec:
  2800. self._warn_filter_objects()
  2801. data = req_data.getvalue()
  2802. resp, read = self._smart_request("git-upload-pack", url, data)
  2803. try:
  2804. resp_proto = Protocol(read, None) # type: ignore
  2805. if new_shallow is None and new_unshallow is None:
  2806. (new_shallow, new_unshallow) = _read_shallow_updates(
  2807. resp_proto.read_pkt_seq()
  2808. )
  2809. _handle_upload_pack_tail(
  2810. resp_proto,
  2811. negotiated_capabilities,
  2812. graph_walker,
  2813. pack_data,
  2814. progress,
  2815. protocol_version=self.protocol_version,
  2816. )
  2817. return FetchPackResult(refs, symrefs, agent, new_shallow, new_unshallow)
  2818. finally:
  2819. resp.close()
  2820. def get_refs(
  2821. self,
  2822. path,
  2823. protocol_version: Optional[int] = None,
  2824. ref_prefix: Optional[list[Ref]] = None,
  2825. ):
  2826. """Retrieve the current refs from a git smart server."""
  2827. url = self._get_url(path)
  2828. refs, _, _, symrefs, peeled = self._discover_references(
  2829. b"git-upload-pack",
  2830. url,
  2831. protocol_version=protocol_version,
  2832. ref_prefix=ref_prefix,
  2833. )
  2834. for refname, refvalue in peeled.items():
  2835. refs[refname + PEELED_TAG_SUFFIX] = refvalue
  2836. return LsRemoteResult(refs, symrefs)
  2837. def get_url(self, path):
  2838. return self._get_url(path).rstrip("/")
  2839. def _get_url(self, path):
  2840. return urljoin(self._base_url, path).rstrip("/") + "/"
  2841. @classmethod
  2842. def from_parsedurl(cls, parsedurl, **kwargs):
  2843. password = parsedurl.password
  2844. if password is not None:
  2845. kwargs["password"] = urlunquote(password)
  2846. username = parsedurl.username
  2847. if username is not None:
  2848. kwargs["username"] = urlunquote(username)
  2849. return cls(urlunparse(parsedurl), **kwargs)
  2850. def __repr__(self) -> str:
  2851. return f"{type(self).__name__}({self._base_url!r}, dumb={self.dumb!r})"
  2852. def _wrap_urllib3_exceptions(func):
  2853. from urllib3.exceptions import ProtocolError
  2854. def wrapper(*args, **kwargs):
  2855. try:
  2856. return func(*args, **kwargs)
  2857. except ProtocolError as error:
  2858. raise GitProtocolError(str(error)) from error
  2859. return wrapper
  2860. class Urllib3HttpGitClient(AbstractHttpGitClient):
  2861. def __init__(
  2862. self,
  2863. base_url,
  2864. dumb=None,
  2865. pool_manager=None,
  2866. config=None,
  2867. username=None,
  2868. password=None,
  2869. timeout=None,
  2870. **kwargs,
  2871. ) -> None:
  2872. self._username = username
  2873. self._password = password
  2874. self._timeout = timeout
  2875. if pool_manager is None:
  2876. self.pool_manager = default_urllib3_manager(
  2877. config, base_url=base_url, timeout=timeout
  2878. )
  2879. else:
  2880. self.pool_manager = pool_manager
  2881. if username is not None:
  2882. # No escaping needed: ":" is not allowed in username:
  2883. # https://tools.ietf.org/html/rfc2617#section-2
  2884. credentials = f"{username}:{password or ''}"
  2885. import urllib3.util
  2886. basic_auth = urllib3.util.make_headers(basic_auth=credentials)
  2887. self.pool_manager.headers.update(basic_auth) # type: ignore
  2888. self.config = config
  2889. super().__init__(base_url=base_url, dumb=dumb, **kwargs)
  2890. def _get_url(self, path):
  2891. if not isinstance(path, str):
  2892. # urllib3.util.url._encode_invalid_chars() converts the path back
  2893. # to bytes using the utf-8 codec.
  2894. path = path.decode("utf-8")
  2895. return urljoin(self._base_url, path).rstrip("/") + "/"
  2896. def _http_request(self, url, headers=None, data=None, raise_for_status=True):
  2897. import urllib3.exceptions
  2898. req_headers = self.pool_manager.headers.copy()
  2899. if headers is not None:
  2900. req_headers.update(headers)
  2901. req_headers["Pragma"] = "no-cache"
  2902. try:
  2903. request_kwargs = {
  2904. "headers": req_headers,
  2905. "preload_content": False,
  2906. }
  2907. if self._timeout is not None:
  2908. request_kwargs["timeout"] = self._timeout
  2909. if data is None:
  2910. resp = self.pool_manager.request("GET", url, **request_kwargs)
  2911. else:
  2912. request_kwargs["body"] = data
  2913. resp = self.pool_manager.request("POST", url, **request_kwargs)
  2914. except urllib3.exceptions.HTTPError as e:
  2915. raise GitProtocolError(str(e)) from e
  2916. if raise_for_status:
  2917. if resp.status == 404:
  2918. raise NotGitRepository
  2919. if resp.status == 401:
  2920. raise HTTPUnauthorized(resp.headers.get("WWW-Authenticate"), url)
  2921. if resp.status == 407:
  2922. raise HTTPProxyUnauthorized(resp.headers.get("Proxy-Authenticate"), url)
  2923. if resp.status != 200:
  2924. raise GitProtocolError(f"unexpected http resp {resp.status} for {url}")
  2925. resp.content_type = resp.headers.get("Content-Type")
  2926. # Check if geturl() is available (urllib3 version >= 1.23)
  2927. try:
  2928. resp_url = resp.geturl()
  2929. except AttributeError:
  2930. # get_redirect_location() is available for urllib3 >= 1.1
  2931. resp.redirect_location = resp.get_redirect_location()
  2932. else:
  2933. resp.redirect_location = resp_url if resp_url != url else ""
  2934. return resp, _wrap_urllib3_exceptions(resp.read)
  2935. HttpGitClient = Urllib3HttpGitClient
  2936. def _win32_url_to_path(parsed) -> str:
  2937. """Convert a file: URL to a path.
  2938. https://datatracker.ietf.org/doc/html/rfc8089
  2939. """
  2940. assert parsed.scheme == "file"
  2941. _, netloc, path, _, _, _ = parsed
  2942. if netloc == "localhost" or not netloc:
  2943. netloc = ""
  2944. elif (
  2945. netloc
  2946. and len(netloc) >= 2
  2947. and netloc[0].isalpha()
  2948. and netloc[1:2] in (":", ":/")
  2949. ):
  2950. # file://C:/foo.bar/baz or file://C://foo.bar//baz
  2951. netloc = netloc[:2]
  2952. else:
  2953. raise NotImplementedError("Non-local file URLs are not supported")
  2954. from nturl2path import url2pathname
  2955. return url2pathname(netloc + path)
  2956. def get_transport_and_path_from_url(
  2957. url: str, config: Optional[Config] = None, operation: Optional[str] = None, **kwargs
  2958. ) -> tuple[GitClient, str]:
  2959. """Obtain a git client from a URL.
  2960. Args:
  2961. url: URL to open (a unicode string)
  2962. config: Optional config object
  2963. operation: Kind of operation that'll be performed; "pull" or "push"
  2964. Keyword Args:
  2965. thin_packs: Whether or not thin packs should be retrieved
  2966. report_activity: Optional callback for reporting transport
  2967. activity.
  2968. Returns:
  2969. Tuple with client instance and relative path.
  2970. """
  2971. if config is not None:
  2972. url = apply_instead_of(config, url, push=(operation == "push"))
  2973. return _get_transport_and_path_from_url(
  2974. url, config=config, operation=operation, **kwargs
  2975. )
  2976. def _get_transport_and_path_from_url(url, config, operation, **kwargs):
  2977. parsed = urlparse(url)
  2978. if parsed.scheme == "git":
  2979. return (TCPGitClient.from_parsedurl(parsed, **kwargs), parsed.path)
  2980. elif parsed.scheme in ("git+ssh", "ssh"):
  2981. return SSHGitClient.from_parsedurl(parsed, config=config, **kwargs), parsed.path
  2982. elif parsed.scheme in ("http", "https"):
  2983. return (
  2984. HttpGitClient.from_parsedurl(parsed, config=config, **kwargs),
  2985. parsed.path,
  2986. )
  2987. elif parsed.scheme == "file":
  2988. if sys.platform == "win32" or os.name == "nt":
  2989. return default_local_git_client_cls(**kwargs), _win32_url_to_path(parsed)
  2990. return (
  2991. default_local_git_client_cls.from_parsedurl(parsed, **kwargs),
  2992. parsed.path,
  2993. )
  2994. raise ValueError(f"unknown scheme '{parsed.scheme}'")
  2995. def parse_rsync_url(location: str) -> tuple[Optional[str], str, str]:
  2996. """Parse a rsync-style URL."""
  2997. if ":" in location and "@" not in location:
  2998. # SSH with no user@, zero or one leading slash.
  2999. (host, path) = location.split(":", 1)
  3000. user = None
  3001. elif ":" in location:
  3002. # SSH with user@host:foo.
  3003. user_host, path = location.split(":", 1)
  3004. if "@" in user_host:
  3005. user, host = user_host.rsplit("@", 1)
  3006. else:
  3007. user = None
  3008. host = user_host
  3009. else:
  3010. raise ValueError("not a valid rsync-style URL")
  3011. return (user, host, path)
  3012. def get_transport_and_path(
  3013. location: str,
  3014. config: Optional[Config] = None,
  3015. operation: Optional[str] = None,
  3016. **kwargs,
  3017. ) -> tuple[GitClient, str]:
  3018. """Obtain a git client from a URL.
  3019. Args:
  3020. location: URL or path (a string)
  3021. config: Optional config object
  3022. operation: Kind of operation that'll be performed; "pull" or "push"
  3023. Keyword Args:
  3024. thin_packs: Whether or not thin packs should be retrieved
  3025. report_activity: Optional callback for reporting transport
  3026. activity.
  3027. Returns:
  3028. Tuple with client instance and relative path.
  3029. """
  3030. if config is not None:
  3031. location = apply_instead_of(config, location, push=(operation == "push"))
  3032. # First, try to parse it as a URL
  3033. try:
  3034. return _get_transport_and_path_from_url(
  3035. location, config=config, operation=operation, **kwargs
  3036. )
  3037. except ValueError:
  3038. pass
  3039. if sys.platform == "win32" and location[0].isalpha() and location[1:3] == ":\\":
  3040. # Windows local path - but check if it's a bundle file first
  3041. if BundleClient._is_bundle_file(location):
  3042. return BundleClient(**kwargs), location
  3043. return default_local_git_client_cls(**kwargs), location
  3044. try:
  3045. (username, hostname, path) = parse_rsync_url(location)
  3046. except ValueError:
  3047. # Check if it's a bundle file before assuming it's a local path
  3048. if BundleClient._is_bundle_file(location):
  3049. return BundleClient(**kwargs), location
  3050. # Otherwise, assume it's a local path.
  3051. return default_local_git_client_cls(**kwargs), location
  3052. else:
  3053. return SSHGitClient(hostname, username=username, config=config, **kwargs), path
  3054. DEFAULT_GIT_CREDENTIALS_PATHS = [
  3055. os.path.expanduser("~/.git-credentials"),
  3056. get_xdg_config_home_path("git", "credentials"),
  3057. ]
  3058. def get_credentials_from_store(
  3059. scheme, hostname, username=None, fnames=DEFAULT_GIT_CREDENTIALS_PATHS
  3060. ):
  3061. for fname in fnames:
  3062. try:
  3063. with open(fname, "rb") as f:
  3064. for line in f:
  3065. parsed_line = urlparse(line.strip())
  3066. if (
  3067. parsed_line.scheme == scheme
  3068. and parsed_line.hostname == hostname
  3069. and (username is None or parsed_line.username == username)
  3070. ):
  3071. return parsed_line.username, parsed_line.password
  3072. except FileNotFoundError:
  3073. # If the file doesn't exist, try the next one.
  3074. continue