client.py 123 KB

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