aot_compiler.c 160 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510351135123513351435153516351735183519352035213522352335243525352635273528352935303531353235333534353535363537353835393540354135423543354435453546354735483549355035513552355335543555355635573558355935603561356235633564356535663567356835693570357135723573357435753576357735783579358035813582358335843585358635873588358935903591359235933594359535963597359835993600360136023603360436053606360736083609361036113612361336143615361636173618361936203621362236233624362536263627362836293630363136323633363436353636363736383639364036413642364336443645364636473648364936503651365236533654365536563657365836593660366136623663366436653666366736683669367036713672367336743675367636773678367936803681368236833684368536863687368836893690369136923693369436953696369736983699370037013702370337043705370637073708370937103711371237133714371537163717371837193720372137223723372437253726372737283729373037313732373337343735373637373738373937403741374237433744374537463747374837493750375137523753375437553756375737583759376037613762376337643765376637673768376937703771377237733774377537763777377837793780378137823783378437853786378737883789379037913792379337943795379637973798379938003801380238033804380538063807380838093810381138123813381438153816381738183819382038213822382338243825382638273828382938303831383238333834383538363837383838393840384138423843384438453846384738483849385038513852385338543855385638573858385938603861386238633864386538663867386838693870387138723873387438753876387738783879388038813882388338843885388638873888388938903891389238933894389538963897389838993900390139023903390439053906390739083909391039113912391339143915391639173918391939203921392239233924392539263927392839293930393139323933393439353936393739383939394039413942394339443945394639473948394939503951395239533954395539563957395839593960396139623963396439653966396739683969397039713972397339743975397639773978397939803981398239833984398539863987398839893990399139923993399439953996399739983999400040014002400340044005400640074008400940104011401240134014401540164017401840194020402140224023402440254026402740284029403040314032403340344035403640374038403940404041404240434044404540464047404840494050405140524053405440554056405740584059406040614062406340644065406640674068406940704071407240734074407540764077407840794080408140824083408440854086408740884089409040914092409340944095409640974098409941004101410241034104410541064107410841094110411141124113411441154116411741184119412041214122412341244125412641274128412941304131413241334134413541364137413841394140414141424143414441454146414741484149415041514152415341544155415641574158415941604161416241634164416541664167416841694170417141724173417441754176417741784179418041814182418341844185418641874188418941904191419241934194419541964197419841994200420142024203420442054206420742084209421042114212421342144215421642174218421942204221422242234224422542264227422842294230423142324233423442354236423742384239424042414242424342444245424642474248424942504251425242534254425542564257425842594260426142624263426442654266
  1. /*
  2. * Copyright (C) 2019 Intel Corporation. All rights reserved.
  3. * SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
  4. */
  5. #include "aot_compiler.h"
  6. #include "aot_emit_compare.h"
  7. #include "aot_emit_conversion.h"
  8. #include "aot_emit_memory.h"
  9. #include "aot_emit_variable.h"
  10. #include "aot_emit_const.h"
  11. #include "aot_emit_exception.h"
  12. #include "aot_emit_numberic.h"
  13. #include "aot_emit_control.h"
  14. #include "aot_emit_function.h"
  15. #include "aot_emit_parametric.h"
  16. #include "aot_emit_table.h"
  17. #include "aot_emit_gc.h"
  18. #include "simd/simd_access_lanes.h"
  19. #include "simd/simd_bitmask_extracts.h"
  20. #include "simd/simd_bit_shifts.h"
  21. #include "simd/simd_bitwise_ops.h"
  22. #include "simd/simd_bool_reductions.h"
  23. #include "simd/simd_comparisons.h"
  24. #include "simd/simd_conversions.h"
  25. #include "simd/simd_construct_values.h"
  26. #include "simd/simd_conversions.h"
  27. #include "simd/simd_floating_point.h"
  28. #include "simd/simd_int_arith.h"
  29. #include "simd/simd_load_store.h"
  30. #include "simd/simd_sat_int_arith.h"
  31. #include "../aot/aot_runtime.h"
  32. #include "../interpreter/wasm_opcode.h"
  33. #include <errno.h>
  34. #if WASM_ENABLE_DEBUG_AOT != 0
  35. #include "debug/dwarf_extractor.h"
  36. #endif
  37. #if WASM_ENABLE_STRINGREF != 0
  38. #include "string_object.h"
  39. #include "aot_emit_stringref.h"
  40. #endif
  41. #define CHECK_BUF(buf, buf_end, length) \
  42. do { \
  43. if (buf + length > buf_end) { \
  44. aot_set_last_error("read leb failed: unexpected end."); \
  45. return false; \
  46. } \
  47. } while (0)
  48. static bool
  49. read_leb(const uint8 *buf, const uint8 *buf_end, uint32 *p_offset,
  50. uint32 maxbits, bool sign, uint64 *p_result)
  51. {
  52. uint64 result = 0;
  53. uint32 shift = 0;
  54. uint32 bcnt = 0;
  55. uint64 byte;
  56. while (true) {
  57. CHECK_BUF(buf, buf_end, 1);
  58. byte = buf[*p_offset];
  59. *p_offset += 1;
  60. result |= ((byte & 0x7f) << shift);
  61. shift += 7;
  62. if ((byte & 0x80) == 0) {
  63. break;
  64. }
  65. bcnt += 1;
  66. }
  67. if (bcnt > (maxbits + 6) / 7) {
  68. aot_set_last_error("read leb failed: "
  69. "integer representation too long");
  70. return false;
  71. }
  72. if (sign && (shift < maxbits) && (byte & 0x40)) {
  73. /* Sign extend */
  74. result |= (~((uint64)0)) << shift;
  75. }
  76. *p_result = result;
  77. return true;
  78. }
  79. /* NOLINTNEXTLINE */
  80. #define read_leb_uint32(p, p_end, res) \
  81. do { \
  82. uint32 off = 0; \
  83. uint64 res64; \
  84. if (!read_leb(p, p_end, &off, 32, false, &res64)) \
  85. return false; \
  86. p += off; \
  87. res = (uint32)res64; \
  88. } while (0)
  89. /* NOLINTNEXTLINE */
  90. #define read_leb_int32(p, p_end, res) \
  91. do { \
  92. uint32 off = 0; \
  93. uint64 res64; \
  94. if (!read_leb(p, p_end, &off, 32, true, &res64)) \
  95. return false; \
  96. p += off; \
  97. res = (int32)res64; \
  98. } while (0)
  99. /* NOLINTNEXTLINE */
  100. #define read_leb_int64(p, p_end, res) \
  101. do { \
  102. uint32 off = 0; \
  103. uint64 res64; \
  104. if (!read_leb(p, p_end, &off, 64, true, &res64)) \
  105. return false; \
  106. p += off; \
  107. res = (int64)res64; \
  108. } while (0)
  109. /**
  110. * Since wamrc uses a full feature Wasm loader,
  111. * add a post-validator here to run checks according
  112. * to options, like enable_tail_call, enable_ref_types,
  113. * and so on.
  114. */
  115. static bool
  116. aot_validate_wasm(AOTCompContext *comp_ctx)
  117. {
  118. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  119. /* Doesn't support multiple tables unless enabling reference type */
  120. if (comp_ctx->comp_data->import_table_count
  121. + comp_ctx->comp_data->table_count
  122. > 1) {
  123. aot_set_last_error("multiple tables");
  124. return false;
  125. }
  126. }
  127. return true;
  128. }
  129. #define COMPILE_ATOMIC_RMW(OP, NAME) \
  130. case WASM_OP_ATOMIC_RMW_I32_##NAME: \
  131. bytes = 4; \
  132. op_type = VALUE_TYPE_I32; \
  133. goto OP_ATOMIC_##OP; \
  134. case WASM_OP_ATOMIC_RMW_I64_##NAME: \
  135. bytes = 8; \
  136. op_type = VALUE_TYPE_I64; \
  137. goto OP_ATOMIC_##OP; \
  138. case WASM_OP_ATOMIC_RMW_I32_##NAME##8_U: \
  139. bytes = 1; \
  140. op_type = VALUE_TYPE_I32; \
  141. goto OP_ATOMIC_##OP; \
  142. case WASM_OP_ATOMIC_RMW_I32_##NAME##16_U: \
  143. bytes = 2; \
  144. op_type = VALUE_TYPE_I32; \
  145. goto OP_ATOMIC_##OP; \
  146. case WASM_OP_ATOMIC_RMW_I64_##NAME##8_U: \
  147. bytes = 1; \
  148. op_type = VALUE_TYPE_I64; \
  149. goto OP_ATOMIC_##OP; \
  150. case WASM_OP_ATOMIC_RMW_I64_##NAME##16_U: \
  151. bytes = 2; \
  152. op_type = VALUE_TYPE_I64; \
  153. goto OP_ATOMIC_##OP; \
  154. case WASM_OP_ATOMIC_RMW_I64_##NAME##32_U: \
  155. bytes = 4; \
  156. op_type = VALUE_TYPE_I64; \
  157. OP_ATOMIC_##OP : bin_op = LLVMAtomicRMWBinOp##OP; \
  158. goto build_atomic_rmw;
  159. uint32
  160. offset_of_local_in_outs_area(AOTCompContext *comp_ctx, unsigned n)
  161. {
  162. AOTCompFrame *frame = comp_ctx->aot_frame;
  163. return frame->cur_frame_size + offset_of_local(comp_ctx, n);
  164. }
  165. static bool
  166. store_value(AOTCompContext *comp_ctx, LLVMValueRef value, uint8 value_type,
  167. LLVMValueRef cur_frame, uint32 offset)
  168. {
  169. LLVMValueRef value_offset, value_addr, value_ptr = NULL, res;
  170. LLVMTypeRef value_ptr_type = NULL;
  171. if (!(value_offset = I32_CONST(offset))) {
  172. aot_set_last_error("llvm build const failed");
  173. return false;
  174. }
  175. if (!(value_addr =
  176. LLVMBuildInBoundsGEP2(comp_ctx->builder, INT8_TYPE, cur_frame,
  177. &value_offset, 1, "value_addr"))) {
  178. aot_set_last_error("llvm build in bounds gep failed");
  179. return false;
  180. }
  181. switch (value_type) {
  182. case VALUE_TYPE_I32:
  183. value_ptr_type = INT32_PTR_TYPE;
  184. break;
  185. case VALUE_TYPE_I64:
  186. value_ptr_type = INT64_PTR_TYPE;
  187. break;
  188. case VALUE_TYPE_F32:
  189. value_ptr_type = F32_PTR_TYPE;
  190. break;
  191. case VALUE_TYPE_F64:
  192. value_ptr_type = F64_PTR_TYPE;
  193. break;
  194. case VALUE_TYPE_V128:
  195. value_ptr_type = V128_PTR_TYPE;
  196. break;
  197. #if WASM_ENABLE_GC != 0
  198. case VALUE_TYPE_GC_REF:
  199. value_ptr_type = GC_REF_PTR_TYPE;
  200. break;
  201. #endif
  202. default:
  203. bh_assert(0);
  204. break;
  205. }
  206. if (!(value_ptr = LLVMBuildBitCast(comp_ctx->builder, value_addr,
  207. value_ptr_type, "value_ptr"))) {
  208. aot_set_last_error("llvm build bit cast failed");
  209. return false;
  210. }
  211. if (!(res = LLVMBuildStore(comp_ctx->builder, value, value_ptr))) {
  212. aot_set_last_error("llvm build store failed");
  213. return false;
  214. }
  215. LLVMSetAlignment(res, 4);
  216. return true;
  217. }
  218. bool
  219. aot_frame_store_value(AOTCompContext *comp_ctx, LLVMValueRef value,
  220. uint8 value_type, LLVMValueRef cur_frame, uint32 offset)
  221. {
  222. return store_value(comp_ctx, value, value_type, cur_frame, offset);
  223. }
  224. static bool
  225. store_ref(AOTCompContext *comp_ctx, uint32 ref, LLVMValueRef cur_frame,
  226. uint32 offset, uint32 nbytes)
  227. {
  228. LLVMValueRef value_ref = NULL, value_offset, value_addr, res;
  229. if (!(value_offset = I32_CONST(offset))) {
  230. aot_set_last_error("llvm build const failed");
  231. return false;
  232. }
  233. if (!(value_addr =
  234. LLVMBuildInBoundsGEP2(comp_ctx->builder, INT8_TYPE, cur_frame,
  235. &value_offset, 1, "value_addr"))) {
  236. aot_set_last_error("llvm build in bounds gep failed");
  237. return false;
  238. }
  239. switch (nbytes) {
  240. case 1:
  241. if (!(value_ref = I8_CONST((uint8)ref))) {
  242. aot_set_last_error("llvm build const failed");
  243. }
  244. break;
  245. case 2:
  246. ref = (ref << 8) | ref;
  247. if (!(value_ref = LLVMConstInt(INT16_TYPE, (uint16)ref, false))) {
  248. aot_set_last_error("llvm build const failed");
  249. return false;
  250. }
  251. if (!(value_addr =
  252. LLVMBuildBitCast(comp_ctx->builder, value_addr,
  253. INT16_PTR_TYPE, "value_addr"))) {
  254. aot_set_last_error("llvm build bit cast failed");
  255. return false;
  256. }
  257. break;
  258. case 4:
  259. ref = (ref << 24) | (ref << 16) | (ref << 8) | ref;
  260. if (!(value_ref = I32_CONST(ref))) {
  261. aot_set_last_error("llvm build const failed");
  262. return false;
  263. }
  264. if (!(value_addr =
  265. LLVMBuildBitCast(comp_ctx->builder, value_addr,
  266. INT32_PTR_TYPE, "value_addr"))) {
  267. aot_set_last_error("llvm build bit cast failed");
  268. return false;
  269. }
  270. break;
  271. default:
  272. bh_assert(0);
  273. break;
  274. }
  275. if (!(res = LLVMBuildStore(comp_ctx->builder, value_ref, value_addr))) {
  276. aot_set_last_error("llvm build store failed");
  277. return false;
  278. }
  279. LLVMSetAlignment(res, 1);
  280. return true;
  281. }
  282. bool
  283. aot_gen_commit_values(AOTCompFrame *frame)
  284. {
  285. AOTCompContext *comp_ctx = frame->comp_ctx;
  286. AOTFuncContext *func_ctx = frame->func_ctx;
  287. AOTValueSlot *p, *end;
  288. LLVMValueRef value;
  289. uint32 n;
  290. /* First, commit reference flags
  291. * For LLVM JIT, iterate all local and stack ref flags
  292. * For AOT, ignore local(params + locals) ref flags */
  293. for (p = comp_ctx->is_jit_mode ? frame->lp
  294. : frame->lp + frame->max_local_cell_num;
  295. p < frame->sp; p++) {
  296. if (!p->dirty)
  297. continue;
  298. n = (uint32)(p - frame->lp);
  299. /* Commit reference flag */
  300. if (comp_ctx->enable_gc) {
  301. switch (p->type) {
  302. case VALUE_TYPE_I32:
  303. case VALUE_TYPE_F32:
  304. case VALUE_TYPE_I1:
  305. if (p->ref != p->committed_ref - 1) {
  306. if (!store_ref(comp_ctx, p->ref, func_ctx->cur_frame,
  307. offset_of_ref(comp_ctx, n), 1))
  308. return false;
  309. p->committed_ref = p->ref + 1;
  310. }
  311. break;
  312. case VALUE_TYPE_I64:
  313. case VALUE_TYPE_F64:
  314. bh_assert(p->ref == (p + 1)->ref);
  315. if (p->ref != p->committed_ref - 1
  316. || p->ref != (p + 1)->committed_ref - 1) {
  317. if (!store_ref(comp_ctx, p->ref, func_ctx->cur_frame,
  318. offset_of_ref(comp_ctx, n), 2))
  319. return false;
  320. p->committed_ref = (p + 1)->committed_ref = p->ref + 1;
  321. }
  322. p++;
  323. break;
  324. case VALUE_TYPE_V128:
  325. bh_assert(p->ref == (p + 1)->ref && p->ref == (p + 2)->ref
  326. && p->ref == (p + 3)->ref);
  327. if (p->ref != p->committed_ref - 1
  328. || p->ref != (p + 1)->committed_ref - 1
  329. || p->ref != (p + 2)->committed_ref - 1
  330. || p->ref != (p + 3)->committed_ref - 1) {
  331. if (!store_ref(comp_ctx, p->ref, func_ctx->cur_frame,
  332. offset_of_ref(comp_ctx, n), 4))
  333. return false;
  334. p->committed_ref = (p + 1)->committed_ref =
  335. (p + 2)->committed_ref = (p + 3)->committed_ref =
  336. p->ref + 1;
  337. }
  338. p += 3;
  339. break;
  340. case REF_TYPE_NULLFUNCREF:
  341. case REF_TYPE_NULLEXTERNREF:
  342. case REF_TYPE_NULLREF:
  343. case REF_TYPE_FUNCREF:
  344. case REF_TYPE_EXTERNREF:
  345. case REF_TYPE_ANYREF:
  346. case REF_TYPE_EQREF:
  347. case REF_TYPE_HT_NULLABLE:
  348. case REF_TYPE_HT_NON_NULLABLE:
  349. case REF_TYPE_I31REF:
  350. case REF_TYPE_STRUCTREF:
  351. case REF_TYPE_ARRAYREF:
  352. #if WASM_ENABLE_STRINGREF != 0
  353. case REF_TYPE_STRINGREF:
  354. case REF_TYPE_STRINGVIEWWTF8:
  355. case REF_TYPE_STRINGVIEWWTF16:
  356. case REF_TYPE_STRINGVIEWITER:
  357. #endif
  358. case VALUE_TYPE_GC_REF:
  359. if (comp_ctx->pointer_size == sizeof(uint64)) {
  360. bh_assert(p->ref == (p + 1)->ref);
  361. if (p->ref != p->committed_ref - 1
  362. || p->ref != (p + 1)->committed_ref - 1) {
  363. if (!store_ref(comp_ctx, p->ref,
  364. func_ctx->cur_frame,
  365. offset_of_ref(comp_ctx, n), 2))
  366. return false;
  367. p->committed_ref = (p + 1)->committed_ref =
  368. p->ref + 1;
  369. }
  370. p++;
  371. }
  372. else {
  373. if (p->ref != p->committed_ref - 1) {
  374. if (!store_ref(comp_ctx, p->ref,
  375. func_ctx->cur_frame,
  376. offset_of_ref(comp_ctx, n), 1))
  377. return false;
  378. p->committed_ref = p->ref + 1;
  379. }
  380. }
  381. break;
  382. default:
  383. bh_assert(0);
  384. break;
  385. }
  386. }
  387. }
  388. /* Second, commit all values */
  389. for (p = frame->lp; p < frame->sp; p++) {
  390. if (!p->dirty)
  391. continue;
  392. p->dirty = 0;
  393. n = (uint32)(p - frame->lp);
  394. /* Commit values */
  395. switch (p->type) {
  396. case VALUE_TYPE_I32:
  397. if (!store_value(comp_ctx, p->value, VALUE_TYPE_I32,
  398. func_ctx->cur_frame,
  399. offset_of_local(comp_ctx, n)))
  400. return false;
  401. break;
  402. case VALUE_TYPE_I64:
  403. (++p)->dirty = 0;
  404. if (!store_value(comp_ctx, p->value, VALUE_TYPE_I64,
  405. func_ctx->cur_frame,
  406. offset_of_local(comp_ctx, n)))
  407. return false;
  408. break;
  409. case VALUE_TYPE_F32:
  410. if (!store_value(comp_ctx, p->value, VALUE_TYPE_F32,
  411. func_ctx->cur_frame,
  412. offset_of_local(comp_ctx, n)))
  413. return false;
  414. break;
  415. case VALUE_TYPE_F64:
  416. (++p)->dirty = 0;
  417. if (!store_value(comp_ctx, p->value, VALUE_TYPE_F64,
  418. func_ctx->cur_frame,
  419. offset_of_local(comp_ctx, n)))
  420. return false;
  421. break;
  422. case VALUE_TYPE_V128:
  423. (++p)->dirty = 0;
  424. (++p)->dirty = 0;
  425. (++p)->dirty = 0;
  426. if (!store_value(comp_ctx, p->value, VALUE_TYPE_V128,
  427. func_ctx->cur_frame,
  428. offset_of_local(comp_ctx, n)))
  429. return false;
  430. break;
  431. case VALUE_TYPE_I1:
  432. if (!(value = LLVMBuildZExt(comp_ctx->builder, p->value,
  433. I32_TYPE, "i32_val"))) {
  434. aot_set_last_error("llvm build bit cast failed");
  435. return false;
  436. }
  437. if (!store_value(comp_ctx, value, VALUE_TYPE_I32,
  438. func_ctx->cur_frame,
  439. offset_of_local(comp_ctx, n)))
  440. return false;
  441. break;
  442. case VALUE_TYPE_FUNCREF:
  443. case VALUE_TYPE_EXTERNREF:
  444. if (comp_ctx->enable_ref_types) {
  445. if (!store_value(comp_ctx, p->value, VALUE_TYPE_I32,
  446. func_ctx->cur_frame,
  447. offset_of_local(comp_ctx, n)))
  448. return false;
  449. }
  450. #if WASM_ENABLE_GC != 0
  451. else if (comp_ctx->enable_gc) {
  452. if (comp_ctx->pointer_size == sizeof(uint64))
  453. (++p)->dirty = 0;
  454. if (!store_value(comp_ctx, p->value, VALUE_TYPE_GC_REF,
  455. func_ctx->cur_frame,
  456. offset_of_local(comp_ctx, n)))
  457. return false;
  458. }
  459. #endif
  460. else {
  461. bh_assert(0);
  462. }
  463. break;
  464. #if WASM_ENABLE_GC != 0
  465. case REF_TYPE_NULLFUNCREF:
  466. case REF_TYPE_NULLEXTERNREF:
  467. case REF_TYPE_NULLREF:
  468. /* case REF_TYPE_FUNCREF: */
  469. /* case REF_TYPE_EXTERNREF: */
  470. case REF_TYPE_ANYREF:
  471. case REF_TYPE_EQREF:
  472. case REF_TYPE_HT_NULLABLE:
  473. case REF_TYPE_HT_NON_NULLABLE:
  474. case REF_TYPE_I31REF:
  475. case REF_TYPE_STRUCTREF:
  476. case REF_TYPE_ARRAYREF:
  477. case VALUE_TYPE_GC_REF:
  478. if (comp_ctx->pointer_size == sizeof(uint64))
  479. (++p)->dirty = 0;
  480. if (!store_value(comp_ctx, p->value, VALUE_TYPE_GC_REF,
  481. func_ctx->cur_frame,
  482. offset_of_local(comp_ctx, n)))
  483. return false;
  484. break;
  485. #endif
  486. default:
  487. bh_assert(0);
  488. break;
  489. }
  490. }
  491. if (comp_ctx->enable_gc) {
  492. end = frame->lp + frame->max_local_cell_num + frame->max_stack_cell_num;
  493. /* Clear reference flags for unused stack slots. */
  494. for (p = frame->sp; p < end; p++) {
  495. bh_assert(!p->ref);
  496. n = (uint32)(p - frame->lp);
  497. /* Commit reference flag. */
  498. if (p->ref != p->committed_ref - 1) {
  499. if (!store_ref(comp_ctx, p->ref, func_ctx->cur_frame,
  500. offset_of_ref(comp_ctx, n), 1))
  501. return false;
  502. p->committed_ref = 1 + p->ref;
  503. }
  504. }
  505. }
  506. return true;
  507. }
  508. bool
  509. aot_gen_commit_sp_ip(AOTCompFrame *frame, bool commit_sp, bool commit_ip)
  510. {
  511. AOTCompContext *comp_ctx = frame->comp_ctx;
  512. AOTFuncContext *func_ctx = frame->func_ctx;
  513. LLVMValueRef cur_frame = func_ctx->cur_frame;
  514. LLVMValueRef value_offset, value_addr, value_ptr, value;
  515. LLVMTypeRef int8_ptr_ptr_type;
  516. uint32 offset_ip, offset_sp, n;
  517. bool is_64bit = (comp_ctx->pointer_size == sizeof(uint64)) ? true : false;
  518. const AOTValueSlot *sp = frame->sp;
  519. const uint8 *ip = frame->frame_ip;
  520. if (!comp_ctx->is_jit_mode) {
  521. offset_ip = frame->comp_ctx->pointer_size * 4;
  522. offset_sp = frame->comp_ctx->pointer_size * 5;
  523. }
  524. else {
  525. offset_ip = offsetof(WASMInterpFrame, ip);
  526. offset_sp = offsetof(WASMInterpFrame, sp);
  527. }
  528. if (commit_ip) {
  529. if (!(value_offset = I32_CONST(offset_ip))) {
  530. aot_set_last_error("llvm build const failed");
  531. return false;
  532. }
  533. if (!(value_addr =
  534. LLVMBuildInBoundsGEP2(comp_ctx->builder, INT8_TYPE, cur_frame,
  535. &value_offset, 1, "ip_addr"))) {
  536. aot_set_last_error("llvm build in bounds gep failed");
  537. return false;
  538. }
  539. if (!(value_ptr = LLVMBuildBitCast(
  540. comp_ctx->builder, value_addr,
  541. is_64bit ? INT64_PTR_TYPE : INT32_PTR_TYPE, "ip_ptr"))) {
  542. aot_set_last_error("llvm build bit cast failed");
  543. return false;
  544. }
  545. if (!comp_ctx->is_jit_mode) {
  546. WASMModule *module = comp_ctx->comp_data->wasm_module;
  547. if (is_64bit)
  548. value = I64_CONST((uint64)(uintptr_t)(ip - module->load_addr));
  549. else
  550. value = I32_CONST((uint32)(uintptr_t)(ip - module->load_addr));
  551. }
  552. else {
  553. if (is_64bit)
  554. value = I64_CONST((uint64)(uintptr_t)ip);
  555. else
  556. value = I32_CONST((uint32)(uintptr_t)ip);
  557. }
  558. if (!value) {
  559. aot_set_last_error("llvm build const failed");
  560. return false;
  561. }
  562. if (!LLVMBuildStore(comp_ctx->builder, value, value_ptr)) {
  563. aot_set_last_error("llvm build store failed");
  564. return false;
  565. }
  566. }
  567. if (commit_sp) {
  568. n = (uint32)(sp - frame->lp);
  569. value = I32_CONST(offset_of_local(comp_ctx, n));
  570. if (!value) {
  571. aot_set_last_error("llvm build const failed");
  572. return false;
  573. }
  574. if (!(value = LLVMBuildInBoundsGEP2(comp_ctx->builder, INT8_TYPE,
  575. cur_frame, &value, 1, "sp"))) {
  576. aot_set_last_error("llvm build in bounds gep failed");
  577. return false;
  578. }
  579. if (!(value_offset = I32_CONST(offset_sp))) {
  580. aot_set_last_error("llvm build const failed");
  581. return false;
  582. }
  583. if (!(value_addr =
  584. LLVMBuildInBoundsGEP2(comp_ctx->builder, INT8_TYPE, cur_frame,
  585. &value_offset, 1, "sp_addr"))) {
  586. aot_set_last_error("llvm build in bounds gep failed");
  587. return false;
  588. }
  589. if (!(int8_ptr_ptr_type = LLVMPointerType(INT8_PTR_TYPE, 0))) {
  590. aot_set_last_error("llvm build pointer type failed");
  591. return false;
  592. }
  593. if (!(value_ptr = LLVMBuildBitCast(comp_ctx->builder, value_addr,
  594. int8_ptr_ptr_type, "sp_ptr"))) {
  595. aot_set_last_error("llvm build bit cast failed");
  596. return false;
  597. }
  598. if (!LLVMBuildStore(comp_ctx->builder, value, value_ptr)) {
  599. aot_set_last_error("llvm build store failed");
  600. return false;
  601. }
  602. }
  603. return true;
  604. }
  605. static uint32
  606. get_cur_frame_size(const AOTCompContext *comp_ctx, uint32 max_local_cell_num,
  607. uint32 max_stack_cell_num)
  608. {
  609. uint32 all_cell_num = max_local_cell_num + max_stack_cell_num;
  610. uint32 frame_size;
  611. if (!comp_ctx->is_jit_mode) {
  612. /* Refer to aot_alloc_frame */
  613. if (!comp_ctx->enable_gc)
  614. frame_size = comp_ctx->pointer_size
  615. * (offsetof(AOTFrame, lp) / sizeof(uintptr_t))
  616. + all_cell_num * 4;
  617. else
  618. frame_size = comp_ctx->pointer_size
  619. * (offsetof(AOTFrame, lp) / sizeof(uintptr_t))
  620. + align_uint(all_cell_num * 5, 4);
  621. }
  622. else {
  623. /* Refer to wasm_interp_interp_frame_size */
  624. if (!comp_ctx->enable_gc)
  625. frame_size = offsetof(WASMInterpFrame, lp) + all_cell_num * 4;
  626. else
  627. frame_size =
  628. offsetof(WASMInterpFrame, lp) + align_uint(all_cell_num * 5, 4);
  629. }
  630. return frame_size;
  631. }
  632. static bool
  633. init_comp_frame(AOTCompContext *comp_ctx, AOTFuncContext *func_ctx,
  634. uint32 func_idx)
  635. {
  636. AOTCompFrame *aot_frame;
  637. AOTFunc *aot_func = func_ctx->aot_func;
  638. AOTFuncType *func_type = aot_func->func_type;
  639. AOTBlock *block = func_ctx->block_stack.block_list_end;
  640. LLVMValueRef local_value;
  641. uint32 max_local_cell_num =
  642. aot_func->param_cell_num + aot_func->local_cell_num;
  643. uint32 max_stack_cell_num = aot_func->max_stack_cell_num;
  644. uint32 all_cell_num = max_local_cell_num + max_stack_cell_num;
  645. uint32 i, n;
  646. uint64 total_size;
  647. uint8 local_type;
  648. /* Free aot_frame if it was allocated previously for
  649. compiling other functions */
  650. if (comp_ctx->aot_frame) {
  651. wasm_runtime_free(comp_ctx->aot_frame);
  652. comp_ctx->aot_frame = NULL;
  653. }
  654. /* Allocate extra 2 cells since some operations may push more
  655. operands than the number calculated in wasm loader, such as
  656. PUSH_F64(F64_CONST(1.0)) in aot_compile_op_f64_promote_f32 */
  657. all_cell_num += 2;
  658. total_size = offsetof(AOTCompFrame, lp)
  659. + (uint64)sizeof(AOTValueSlot) * all_cell_num;
  660. if (total_size > UINT32_MAX
  661. || !(comp_ctx->aot_frame = aot_frame =
  662. wasm_runtime_malloc((uint32)total_size))) {
  663. aot_set_last_error("allocate memory failed.");
  664. return false;
  665. }
  666. memset(aot_frame, 0, (uint32)total_size);
  667. aot_frame->comp_ctx = comp_ctx;
  668. aot_frame->func_ctx = func_ctx;
  669. aot_frame->max_local_cell_num = max_local_cell_num;
  670. aot_frame->max_stack_cell_num = max_stack_cell_num;
  671. aot_frame->cur_frame_size =
  672. get_cur_frame_size(comp_ctx, max_local_cell_num, max_stack_cell_num);
  673. aot_frame->sp = aot_frame->lp + max_local_cell_num;
  674. /* Init the frame_sp_begin and frame_sp_max_reached
  675. of the function block */
  676. block->frame_sp_begin = block->frame_sp_max_reached = aot_frame->sp;
  677. n = 0;
  678. /* Set all params dirty since they were set to llvm value but
  679. haven't been committed to the AOT/JIT stack frame */
  680. for (i = 0; i < func_type->param_count; i++) {
  681. local_type = func_type->types[i];
  682. local_value = LLVMGetParam(func_ctx->func, i + 1);
  683. switch (local_type) {
  684. case VALUE_TYPE_I32:
  685. set_local_i32(comp_ctx->aot_frame, n, local_value);
  686. n++;
  687. break;
  688. case VALUE_TYPE_I64:
  689. set_local_i64(comp_ctx->aot_frame, n, local_value);
  690. n += 2;
  691. break;
  692. case VALUE_TYPE_F32:
  693. set_local_f32(comp_ctx->aot_frame, n, local_value);
  694. n++;
  695. break;
  696. case VALUE_TYPE_F64:
  697. set_local_f64(comp_ctx->aot_frame, n, local_value);
  698. n += 2;
  699. break;
  700. case VALUE_TYPE_V128:
  701. set_local_v128(comp_ctx->aot_frame, n, local_value);
  702. n += 4;
  703. break;
  704. case VALUE_TYPE_FUNCREF:
  705. case VALUE_TYPE_EXTERNREF:
  706. {
  707. if (comp_ctx->enable_ref_types) {
  708. set_local_ref(comp_ctx->aot_frame, n, local_value,
  709. local_type);
  710. n++;
  711. }
  712. #if WASM_ENABLE_GC != 0
  713. else if (comp_ctx->enable_gc) {
  714. set_local_gc_ref(comp_ctx->aot_frame, n, local_value,
  715. VALUE_TYPE_GC_REF);
  716. n += comp_ctx->pointer_size / sizeof(uint32);
  717. }
  718. #endif
  719. else {
  720. bh_assert(0);
  721. }
  722. break;
  723. }
  724. #if WASM_ENABLE_GC != 0
  725. case REF_TYPE_NULLFUNCREF:
  726. case REF_TYPE_NULLEXTERNREF:
  727. case REF_TYPE_NULLREF:
  728. /* case REF_TYPE_FUNCREF: */
  729. /* case REF_TYPE_EXTERNREF: */
  730. case REF_TYPE_ANYREF:
  731. case REF_TYPE_EQREF:
  732. case REF_TYPE_HT_NULLABLE:
  733. case REF_TYPE_HT_NON_NULLABLE:
  734. case REF_TYPE_I31REF:
  735. case REF_TYPE_STRUCTREF:
  736. case REF_TYPE_ARRAYREF:
  737. #if WASM_ENABLE_STRINGREF != 0
  738. case REF_TYPE_STRINGREF:
  739. case REF_TYPE_STRINGVIEWWTF8:
  740. case REF_TYPE_STRINGVIEWWTF16:
  741. case REF_TYPE_STRINGVIEWITER:
  742. #endif
  743. bh_assert(comp_ctx->enable_gc);
  744. set_local_gc_ref(comp_ctx->aot_frame, n, local_value,
  745. VALUE_TYPE_GC_REF);
  746. n += comp_ctx->pointer_size / sizeof(uint32);
  747. break;
  748. #endif
  749. default:
  750. bh_assert(0);
  751. break;
  752. }
  753. }
  754. /* TODO: re-calculate param_cell_num according to the build target
  755. after creating comp_ctx */
  756. /* bh_assert(n == aot_func->param_cell_num); */
  757. /* Set all locals dirty since they were set to llvm value but
  758. haven't been committed to the AOT/JIT stack frame */
  759. for (i = 0; i < aot_func->local_count; i++) {
  760. local_type = aot_func->local_types_wp[i];
  761. switch (local_type) {
  762. case VALUE_TYPE_I32:
  763. set_local_i32(comp_ctx->aot_frame, n, I32_ZERO);
  764. n++;
  765. break;
  766. case VALUE_TYPE_I64:
  767. set_local_i64(comp_ctx->aot_frame, n, I64_ZERO);
  768. n += 2;
  769. break;
  770. case VALUE_TYPE_F32:
  771. set_local_f32(comp_ctx->aot_frame, n, F32_ZERO);
  772. n++;
  773. break;
  774. case VALUE_TYPE_F64:
  775. set_local_f64(comp_ctx->aot_frame, n, F64_ZERO);
  776. n += 2;
  777. break;
  778. case VALUE_TYPE_V128:
  779. set_local_v128(comp_ctx->aot_frame, n, V128_f64x2_ZERO);
  780. n += 4;
  781. break;
  782. case VALUE_TYPE_FUNCREF:
  783. case VALUE_TYPE_EXTERNREF:
  784. {
  785. if (comp_ctx->enable_ref_types) {
  786. set_local_ref(comp_ctx->aot_frame, n, I32_ZERO, local_type);
  787. n++;
  788. }
  789. #if WASM_ENABLE_GC != 0
  790. else if (comp_ctx->enable_gc) {
  791. set_local_gc_ref(comp_ctx->aot_frame, n, GC_REF_NULL,
  792. VALUE_TYPE_GC_REF);
  793. n += comp_ctx->pointer_size / sizeof(uint32);
  794. }
  795. #endif
  796. else {
  797. bh_assert(0);
  798. }
  799. break;
  800. }
  801. #if WASM_ENABLE_GC != 0
  802. case REF_TYPE_NULLFUNCREF:
  803. case REF_TYPE_NULLEXTERNREF:
  804. case REF_TYPE_NULLREF:
  805. /* case REF_TYPE_FUNCREF: */
  806. /* case REF_TYPE_EXTERNREF: */
  807. case REF_TYPE_ANYREF:
  808. case REF_TYPE_EQREF:
  809. case REF_TYPE_HT_NULLABLE:
  810. case REF_TYPE_HT_NON_NULLABLE:
  811. case REF_TYPE_I31REF:
  812. case REF_TYPE_STRUCTREF:
  813. case REF_TYPE_ARRAYREF:
  814. #if WASM_ENABLE_STRINGREF != 0
  815. case REF_TYPE_STRINGREF:
  816. case REF_TYPE_STRINGVIEWWTF8:
  817. case REF_TYPE_STRINGVIEWWTF16:
  818. case REF_TYPE_STRINGVIEWITER:
  819. #endif
  820. bh_assert(comp_ctx->enable_gc);
  821. set_local_gc_ref(comp_ctx->aot_frame, n, GC_REF_NULL,
  822. VALUE_TYPE_GC_REF);
  823. n += comp_ctx->pointer_size / sizeof(uint32);
  824. break;
  825. #endif
  826. default:
  827. bh_assert(0);
  828. break;
  829. }
  830. }
  831. /* TODO: re-calculate local_cell_num according to the build target
  832. after creating comp_ctx */
  833. /* bh_assert(n == aot_func->param_cell_num + aot_func->local_cell_num); */
  834. /* No need to initialize aot_frame all cells' committed_ref flags
  835. and all stack cells' ref flags since they have been initialized
  836. as 0 (uncommitted and not-reference) by the memset above */
  837. return true;
  838. }
  839. static bool
  840. aot_compile_func(AOTCompContext *comp_ctx, uint32 func_index)
  841. {
  842. AOTFuncContext *func_ctx = comp_ctx->func_ctxes[func_index];
  843. uint8 *frame_ip = func_ctx->aot_func->code, opcode, *p_f32, *p_f64;
  844. uint8 *frame_ip_end = frame_ip + func_ctx->aot_func->code_size;
  845. uint8 *param_types = NULL;
  846. uint8 *result_types = NULL;
  847. uint8 value_type;
  848. uint16 param_count;
  849. uint16 result_count;
  850. uint32 br_depth, *br_depths, br_count;
  851. uint32 func_idx, type_idx, mem_idx, local_idx, global_idx, i;
  852. uint32 bytes = 4, align, offset;
  853. uint32 type_index;
  854. bool sign = true;
  855. int32 i32_const;
  856. int64 i64_const;
  857. float32 f32_const;
  858. float64 f64_const;
  859. AOTFuncType *func_type = NULL;
  860. #if WASM_ENABLE_DEBUG_AOT != 0
  861. LLVMMetadataRef location;
  862. #endif
  863. if (comp_ctx->enable_aux_stack_frame) {
  864. if (!init_comp_frame(comp_ctx, func_ctx, func_index)) {
  865. return false;
  866. }
  867. }
  868. /* Start to translate the opcodes */
  869. LLVMPositionBuilderAtEnd(
  870. comp_ctx->builder,
  871. func_ctx->block_stack.block_list_head->llvm_entry_block);
  872. while (frame_ip < frame_ip_end) {
  873. opcode = *frame_ip++;
  874. if (comp_ctx->aot_frame) {
  875. comp_ctx->aot_frame->frame_ip = frame_ip - 1;
  876. }
  877. #if WASM_ENABLE_DEBUG_AOT != 0
  878. location = dwarf_gen_location(
  879. comp_ctx, func_ctx,
  880. (frame_ip - 1) - comp_ctx->comp_data->wasm_module->buf_code);
  881. if (location != NULL) {
  882. LLVMSetCurrentDebugLocation2(comp_ctx->builder, location);
  883. }
  884. #endif
  885. switch (opcode) {
  886. case WASM_OP_UNREACHABLE:
  887. if (!aot_compile_op_unreachable(comp_ctx, func_ctx, &frame_ip))
  888. return false;
  889. break;
  890. case WASM_OP_NOP:
  891. break;
  892. case WASM_OP_BLOCK:
  893. case WASM_OP_LOOP:
  894. case WASM_OP_IF:
  895. {
  896. value_type = *frame_ip++;
  897. if (value_type == VALUE_TYPE_I32 || value_type == VALUE_TYPE_I64
  898. || value_type == VALUE_TYPE_F32
  899. || value_type == VALUE_TYPE_F64
  900. || value_type == VALUE_TYPE_V128
  901. || value_type == VALUE_TYPE_VOID
  902. || (comp_ctx->enable_ref_types
  903. && (value_type == VALUE_TYPE_FUNCREF
  904. || value_type == VALUE_TYPE_EXTERNREF))
  905. || (comp_ctx->enable_gc /* single byte type */
  906. && aot_is_type_gc_reftype(value_type))) {
  907. param_count = 0;
  908. param_types = NULL;
  909. if (value_type == VALUE_TYPE_VOID) {
  910. result_count = 0;
  911. result_types = NULL;
  912. }
  913. else {
  914. if (comp_ctx->enable_gc
  915. && aot_is_type_gc_reftype(value_type))
  916. value_type = VALUE_TYPE_GC_REF;
  917. result_count = 1;
  918. result_types = &value_type;
  919. }
  920. }
  921. else {
  922. frame_ip--;
  923. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  924. func_type =
  925. (AOTFuncType *)comp_ctx->comp_data->types[type_index];
  926. param_count = func_type->param_count;
  927. param_types = func_type->types;
  928. result_count = func_type->result_count;
  929. result_types = func_type->types + param_count;
  930. }
  931. if (!aot_compile_op_block(
  932. comp_ctx, func_ctx, &frame_ip, frame_ip_end,
  933. (uint32)(LABEL_TYPE_BLOCK + opcode - WASM_OP_BLOCK),
  934. param_count, param_types, result_count, result_types))
  935. return false;
  936. break;
  937. }
  938. case EXT_OP_BLOCK:
  939. case EXT_OP_LOOP:
  940. case EXT_OP_IF:
  941. {
  942. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  943. func_type =
  944. (AOTFuncType *)comp_ctx->comp_data->types[type_index];
  945. param_count = func_type->param_count;
  946. param_types = func_type->types;
  947. result_count = func_type->result_count;
  948. result_types = func_type->types + param_count;
  949. if (!aot_compile_op_block(
  950. comp_ctx, func_ctx, &frame_ip, frame_ip_end,
  951. (uint32)(LABEL_TYPE_BLOCK + opcode - EXT_OP_BLOCK),
  952. param_count, param_types, result_count, result_types))
  953. return false;
  954. break;
  955. }
  956. case WASM_OP_ELSE:
  957. if (!aot_compile_op_else(comp_ctx, func_ctx, &frame_ip))
  958. return false;
  959. break;
  960. case WASM_OP_END:
  961. if (!aot_compile_op_end(comp_ctx, func_ctx, &frame_ip))
  962. return false;
  963. break;
  964. case WASM_OP_BR:
  965. {
  966. read_leb_uint32(frame_ip, frame_ip_end, br_depth);
  967. if (!aot_compile_op_br(comp_ctx, func_ctx, br_depth, &frame_ip))
  968. return false;
  969. break;
  970. }
  971. case WASM_OP_BR_IF:
  972. {
  973. read_leb_uint32(frame_ip, frame_ip_end, br_depth);
  974. if (!aot_compile_op_br_if(comp_ctx, func_ctx, br_depth,
  975. &frame_ip))
  976. return false;
  977. break;
  978. }
  979. case WASM_OP_BR_TABLE:
  980. {
  981. read_leb_uint32(frame_ip, frame_ip_end, br_count);
  982. if (!(br_depths = wasm_runtime_malloc((uint32)sizeof(uint32)
  983. * (br_count + 1)))) {
  984. aot_set_last_error("allocate memory failed.");
  985. goto fail;
  986. }
  987. #if WASM_ENABLE_FAST_INTERP != 0
  988. for (i = 0; i <= br_count; i++)
  989. read_leb_uint32(frame_ip, frame_ip_end, br_depths[i]);
  990. #else
  991. for (i = 0; i <= br_count; i++)
  992. br_depths[i] = *frame_ip++;
  993. #endif
  994. if (!aot_compile_op_br_table(comp_ctx, func_ctx, br_depths,
  995. br_count, &frame_ip)) {
  996. wasm_runtime_free(br_depths);
  997. return false;
  998. }
  999. wasm_runtime_free(br_depths);
  1000. break;
  1001. }
  1002. #if WASM_ENABLE_FAST_INTERP == 0
  1003. case EXT_OP_BR_TABLE_CACHE:
  1004. {
  1005. BrTableCache *node = bh_list_first_elem(
  1006. comp_ctx->comp_data->wasm_module->br_table_cache_list);
  1007. BrTableCache *node_next;
  1008. const uint8 *frame_ip_org = frame_ip - 1;
  1009. read_leb_uint32(frame_ip, frame_ip_end, br_count);
  1010. while (node) {
  1011. node_next = bh_list_elem_next(node);
  1012. if (node->br_table_op_addr == frame_ip_org) {
  1013. br_depths = node->br_depths;
  1014. if (!aot_compile_op_br_table(comp_ctx, func_ctx,
  1015. br_depths, br_count,
  1016. &frame_ip)) {
  1017. return false;
  1018. }
  1019. break;
  1020. }
  1021. node = node_next;
  1022. }
  1023. bh_assert(node);
  1024. break;
  1025. }
  1026. #endif
  1027. case WASM_OP_RETURN:
  1028. if (!aot_compile_op_return(comp_ctx, func_ctx, &frame_ip))
  1029. return false;
  1030. break;
  1031. case WASM_OP_CALL:
  1032. {
  1033. read_leb_uint32(frame_ip, frame_ip_end, func_idx);
  1034. if (!aot_compile_op_call(comp_ctx, func_ctx, func_idx, false))
  1035. return false;
  1036. break;
  1037. }
  1038. case WASM_OP_CALL_INDIRECT:
  1039. {
  1040. uint32 tbl_idx;
  1041. read_leb_uint32(frame_ip, frame_ip_end, type_idx);
  1042. if (comp_ctx->enable_gc || comp_ctx->enable_ref_types) {
  1043. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  1044. }
  1045. else {
  1046. frame_ip++;
  1047. tbl_idx = 0;
  1048. }
  1049. if (!aot_compile_op_call_indirect(comp_ctx, func_ctx, type_idx,
  1050. tbl_idx))
  1051. return false;
  1052. break;
  1053. }
  1054. #if WASM_ENABLE_TAIL_CALL != 0
  1055. case WASM_OP_RETURN_CALL:
  1056. {
  1057. if (!comp_ctx->enable_tail_call) {
  1058. aot_set_last_error("unsupported opcode");
  1059. return false;
  1060. }
  1061. read_leb_uint32(frame_ip, frame_ip_end, func_idx);
  1062. if (!aot_compile_op_call(comp_ctx, func_ctx, func_idx, true))
  1063. return false;
  1064. if (!aot_compile_op_return(comp_ctx, func_ctx, &frame_ip))
  1065. return false;
  1066. break;
  1067. }
  1068. case WASM_OP_RETURN_CALL_INDIRECT:
  1069. {
  1070. uint32 tbl_idx;
  1071. if (!comp_ctx->enable_tail_call) {
  1072. aot_set_last_error("unsupported opcode");
  1073. return false;
  1074. }
  1075. read_leb_uint32(frame_ip, frame_ip_end, type_idx);
  1076. if (comp_ctx->enable_gc || comp_ctx->enable_ref_types) {
  1077. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  1078. }
  1079. else {
  1080. frame_ip++;
  1081. tbl_idx = 0;
  1082. }
  1083. if (!aot_compile_op_call_indirect(comp_ctx, func_ctx, type_idx,
  1084. tbl_idx))
  1085. return false;
  1086. if (!aot_compile_op_return(comp_ctx, func_ctx, &frame_ip))
  1087. return false;
  1088. break;
  1089. }
  1090. #endif /* end of WASM_ENABLE_TAIL_CALL */
  1091. case WASM_OP_DROP:
  1092. if (!aot_compile_op_drop(comp_ctx, func_ctx, true))
  1093. return false;
  1094. break;
  1095. case WASM_OP_DROP_64:
  1096. if (!aot_compile_op_drop(comp_ctx, func_ctx, false))
  1097. return false;
  1098. break;
  1099. case WASM_OP_SELECT:
  1100. if (!aot_compile_op_select(comp_ctx, func_ctx, true))
  1101. return false;
  1102. break;
  1103. case WASM_OP_SELECT_64:
  1104. if (!aot_compile_op_select(comp_ctx, func_ctx, false))
  1105. return false;
  1106. break;
  1107. #if WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0
  1108. case WASM_OP_SELECT_T:
  1109. {
  1110. uint32 vec_len;
  1111. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1112. goto unsupport_gc_and_ref_types;
  1113. }
  1114. read_leb_uint32(frame_ip, frame_ip_end, vec_len);
  1115. bh_assert(vec_len == 1);
  1116. (void)vec_len;
  1117. type_idx = *frame_ip++;
  1118. if (!aot_compile_op_select(
  1119. comp_ctx, func_ctx,
  1120. (type_idx != VALUE_TYPE_I64)
  1121. && (type_idx != VALUE_TYPE_F64)
  1122. #if WASM_ENABLE_GC != 0
  1123. && !(comp_ctx->enable_gc
  1124. && comp_ctx->pointer_size == sizeof(uint64)
  1125. && wasm_is_type_reftype(type_idx))
  1126. #endif
  1127. ))
  1128. return false;
  1129. break;
  1130. }
  1131. case WASM_OP_TABLE_GET:
  1132. {
  1133. uint32 tbl_idx;
  1134. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1135. goto unsupport_gc_and_ref_types;
  1136. }
  1137. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  1138. if (!aot_compile_op_table_get(comp_ctx, func_ctx, tbl_idx))
  1139. return false;
  1140. break;
  1141. }
  1142. case WASM_OP_TABLE_SET:
  1143. {
  1144. uint32 tbl_idx;
  1145. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1146. goto unsupport_gc_and_ref_types;
  1147. }
  1148. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  1149. if (!aot_compile_op_table_set(comp_ctx, func_ctx, tbl_idx))
  1150. return false;
  1151. break;
  1152. }
  1153. case WASM_OP_REF_NULL:
  1154. {
  1155. uint32 type;
  1156. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1157. goto unsupport_gc_and_ref_types;
  1158. }
  1159. read_leb_uint32(frame_ip, frame_ip_end, type);
  1160. if (!aot_compile_op_ref_null(comp_ctx, func_ctx))
  1161. return false;
  1162. (void)type;
  1163. break;
  1164. }
  1165. case WASM_OP_REF_IS_NULL:
  1166. {
  1167. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1168. goto unsupport_gc_and_ref_types;
  1169. }
  1170. if (!aot_compile_op_ref_is_null(comp_ctx, func_ctx))
  1171. return false;
  1172. break;
  1173. }
  1174. case WASM_OP_REF_FUNC:
  1175. {
  1176. if (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc) {
  1177. goto unsupport_gc_and_ref_types;
  1178. }
  1179. read_leb_uint32(frame_ip, frame_ip_end, func_idx);
  1180. if (!aot_compile_op_ref_func(comp_ctx, func_ctx, func_idx))
  1181. return false;
  1182. break;
  1183. }
  1184. #endif /* end of WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0 */
  1185. #if WASM_ENABLE_GC != 0
  1186. case WASM_OP_CALL_REF:
  1187. {
  1188. if (!comp_ctx->enable_gc) {
  1189. goto unsupport_gc;
  1190. }
  1191. read_leb_uint32(frame_ip, frame_ip_end, type_idx);
  1192. if (!aot_compile_op_call_ref(comp_ctx, func_ctx, type_idx,
  1193. false))
  1194. return false;
  1195. break;
  1196. }
  1197. case WASM_OP_RETURN_CALL_REF:
  1198. {
  1199. if (!comp_ctx->enable_gc) {
  1200. goto unsupport_gc;
  1201. }
  1202. read_leb_uint32(frame_ip, frame_ip_end, type_idx);
  1203. if (!aot_compile_op_call_ref(comp_ctx, func_ctx, type_idx,
  1204. true))
  1205. return false;
  1206. if (!aot_compile_op_return(comp_ctx, func_ctx, &frame_ip))
  1207. return false;
  1208. break;
  1209. }
  1210. case WASM_OP_REF_EQ:
  1211. if (!comp_ctx->enable_gc) {
  1212. goto unsupport_gc;
  1213. }
  1214. if (!aot_compile_op_ref_eq(comp_ctx, func_ctx))
  1215. return false;
  1216. break;
  1217. case WASM_OP_REF_AS_NON_NULL:
  1218. if (!comp_ctx->enable_gc) {
  1219. goto unsupport_gc;
  1220. }
  1221. if (!aot_compile_op_ref_as_non_null(comp_ctx, func_ctx))
  1222. return false;
  1223. break;
  1224. case WASM_OP_BR_ON_NULL:
  1225. {
  1226. if (!comp_ctx->enable_gc) {
  1227. goto unsupport_gc;
  1228. }
  1229. read_leb_uint32(frame_ip, frame_ip_end, br_depth);
  1230. if (!aot_compile_op_br_on_null(comp_ctx, func_ctx, br_depth,
  1231. &frame_ip))
  1232. return false;
  1233. break;
  1234. }
  1235. case WASM_OP_BR_ON_NON_NULL:
  1236. {
  1237. if (!comp_ctx->enable_gc) {
  1238. goto unsupport_gc;
  1239. }
  1240. read_leb_uint32(frame_ip, frame_ip_end, br_depth);
  1241. if (!aot_compile_op_br_on_non_null(comp_ctx, func_ctx, br_depth,
  1242. &frame_ip))
  1243. return false;
  1244. break;
  1245. }
  1246. case WASM_OP_GC_PREFIX:
  1247. {
  1248. uint32 opcode1, field_idx, data_seg_idx, array_len;
  1249. if (!comp_ctx->enable_gc) {
  1250. goto unsupport_gc;
  1251. }
  1252. read_leb_uint32(frame_ip, frame_ip_end, opcode1);
  1253. /* opcode1 was checked in loader and is no larger than
  1254. UINT8_MAX */
  1255. opcode = (uint8)opcode1;
  1256. switch (opcode) {
  1257. case WASM_OP_STRUCT_NEW:
  1258. case WASM_OP_STRUCT_NEW_DEFAULT:
  1259. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1260. if (!aot_compile_op_struct_new(
  1261. comp_ctx, func_ctx, type_index,
  1262. opcode == WASM_OP_STRUCT_NEW_DEFAULT))
  1263. return false;
  1264. break;
  1265. case WASM_OP_STRUCT_GET:
  1266. case WASM_OP_STRUCT_GET_S:
  1267. case WASM_OP_STRUCT_GET_U:
  1268. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1269. read_leb_uint32(frame_ip, frame_ip_end, field_idx);
  1270. if (!aot_compile_op_struct_get(
  1271. comp_ctx, func_ctx, type_index, field_idx,
  1272. opcode == WASM_OP_STRUCT_GET_S))
  1273. return false;
  1274. break;
  1275. case WASM_OP_STRUCT_SET:
  1276. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1277. read_leb_uint32(frame_ip, frame_ip_end, field_idx);
  1278. if (!aot_compile_op_struct_set(comp_ctx, func_ctx,
  1279. type_index, field_idx))
  1280. return false;
  1281. break;
  1282. case WASM_OP_ARRAY_NEW:
  1283. case WASM_OP_ARRAY_NEW_DEFAULT:
  1284. case WASM_OP_ARRAY_NEW_FIXED:
  1285. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1286. if (opcode == WASM_OP_ARRAY_NEW_FIXED)
  1287. read_leb_uint32(frame_ip, frame_ip_end, array_len);
  1288. else
  1289. array_len = 0;
  1290. if (!aot_compile_op_array_new(
  1291. comp_ctx, func_ctx, type_index,
  1292. opcode == WASM_OP_ARRAY_NEW_DEFAULT,
  1293. opcode == WASM_OP_ARRAY_NEW_FIXED, array_len))
  1294. return false;
  1295. break;
  1296. case WASM_OP_ARRAY_NEW_DATA:
  1297. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1298. read_leb_uint32(frame_ip, frame_ip_end, data_seg_idx);
  1299. if (!aot_compile_op_array_new_data(
  1300. comp_ctx, func_ctx, type_index, data_seg_idx))
  1301. return false;
  1302. break;
  1303. case WASM_OP_ARRAY_NEW_ELEM:
  1304. /* TODO */
  1305. aot_set_last_error("unsupported opcode");
  1306. return false;
  1307. case WASM_OP_ARRAY_GET:
  1308. case WASM_OP_ARRAY_GET_S:
  1309. case WASM_OP_ARRAY_GET_U:
  1310. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1311. if (!aot_compile_op_array_get(
  1312. comp_ctx, func_ctx, type_index,
  1313. opcode == WASM_OP_ARRAY_GET_S))
  1314. return false;
  1315. break;
  1316. case WASM_OP_ARRAY_SET:
  1317. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1318. if (!aot_compile_op_array_set(comp_ctx, func_ctx,
  1319. type_index))
  1320. return false;
  1321. break;
  1322. case WASM_OP_ARRAY_FILL:
  1323. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1324. if (!aot_compile_op_array_fill(comp_ctx, func_ctx,
  1325. type_index))
  1326. return false;
  1327. break;
  1328. case WASM_OP_ARRAY_COPY:
  1329. {
  1330. uint32 src_type_index;
  1331. read_leb_uint32(frame_ip, frame_ip_end, type_index);
  1332. read_leb_uint32(frame_ip, frame_ip_end, src_type_index);
  1333. if (!aot_compile_op_array_copy(
  1334. comp_ctx, func_ctx, type_index, src_type_index))
  1335. return false;
  1336. break;
  1337. }
  1338. case WASM_OP_ARRAY_LEN:
  1339. if (!aot_compile_op_array_len(comp_ctx, func_ctx))
  1340. return false;
  1341. break;
  1342. case WASM_OP_REF_I31:
  1343. if (!aot_compile_op_i31_new(comp_ctx, func_ctx))
  1344. return false;
  1345. break;
  1346. case WASM_OP_I31_GET_S:
  1347. case WASM_OP_I31_GET_U:
  1348. if (!aot_compile_op_i31_get(
  1349. comp_ctx, func_ctx,
  1350. opcode == WASM_OP_I31_GET_S ? true : false))
  1351. return false;
  1352. break;
  1353. case WASM_OP_REF_TEST:
  1354. case WASM_OP_REF_TEST_NULLABLE:
  1355. {
  1356. int32 heap_type;
  1357. read_leb_int32(frame_ip, frame_ip_end, heap_type);
  1358. if (!aot_compile_op_ref_test(
  1359. comp_ctx, func_ctx, heap_type,
  1360. opcode == WASM_OP_REF_TEST_NULLABLE ? true
  1361. : false))
  1362. return false;
  1363. break;
  1364. }
  1365. case WASM_OP_REF_CAST:
  1366. case WASM_OP_REF_CAST_NULLABLE:
  1367. {
  1368. int32 heap_type;
  1369. read_leb_int32(frame_ip, frame_ip_end, heap_type);
  1370. if (!aot_compile_op_ref_cast(
  1371. comp_ctx, func_ctx, heap_type,
  1372. opcode == WASM_OP_REF_CAST_NULLABLE ? true
  1373. : false))
  1374. return false;
  1375. break;
  1376. }
  1377. case WASM_OP_BR_ON_CAST:
  1378. case WASM_OP_BR_ON_CAST_FAIL:
  1379. {
  1380. uint8 castflags;
  1381. int32 heap_type, dst_heap_type;
  1382. CHECK_BUF(frame_ip, frame_ip_end, 1);
  1383. castflags = *frame_ip++;
  1384. read_leb_uint32(frame_ip, frame_ip_end, br_depth);
  1385. read_leb_int32(frame_ip, frame_ip_end, heap_type);
  1386. read_leb_int32(frame_ip, frame_ip_end, dst_heap_type);
  1387. /*
  1388. * castflags should be 0~3:
  1389. * 0: (non-null, non-null)
  1390. * 1: (null, non-null)
  1391. * 2: (non-null, null)
  1392. * 3: (null, null)
  1393. * The nullability of source type has been checked in
  1394. * wasm loader, here we just need the dst nullability
  1395. */
  1396. if (!aot_compile_op_br_on_cast(
  1397. comp_ctx, func_ctx, dst_heap_type,
  1398. castflags & 0x02,
  1399. opcode == WASM_OP_BR_ON_CAST_FAIL, br_depth,
  1400. &frame_ip))
  1401. return false;
  1402. (void)heap_type;
  1403. break;
  1404. }
  1405. case WASM_OP_ANY_CONVERT_EXTERN:
  1406. if (!aot_compile_op_extern_internalize(comp_ctx,
  1407. func_ctx))
  1408. return false;
  1409. break;
  1410. case WASM_OP_EXTERN_CONVERT_ANY:
  1411. if (!aot_compile_op_extern_externalize(comp_ctx,
  1412. func_ctx))
  1413. return false;
  1414. break;
  1415. #if WASM_ENABLE_STRINGREF != 0
  1416. case WASM_OP_STRING_NEW_UTF8:
  1417. case WASM_OP_STRING_NEW_WTF16:
  1418. case WASM_OP_STRING_NEW_LOSSY_UTF8:
  1419. case WASM_OP_STRING_NEW_WTF8:
  1420. {
  1421. EncodingFlag flag = WTF8;
  1422. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1423. bh_assert(mem_idx == 0);
  1424. if (opcode == WASM_OP_STRING_NEW_WTF16) {
  1425. flag = WTF16;
  1426. }
  1427. else if (opcode == WASM_OP_STRING_NEW_UTF8) {
  1428. flag = UTF8;
  1429. }
  1430. else if (opcode == WASM_OP_STRING_NEW_LOSSY_UTF8) {
  1431. flag = LOSSY_UTF8;
  1432. }
  1433. else if (opcode == WASM_OP_STRING_NEW_WTF8) {
  1434. flag = WTF8;
  1435. }
  1436. if (!aot_compile_op_string_new(comp_ctx, func_ctx,
  1437. flag))
  1438. return false;
  1439. break;
  1440. }
  1441. case WASM_OP_STRING_CONST:
  1442. {
  1443. uint32 contents;
  1444. read_leb_uint32(frame_ip, frame_ip_end, contents);
  1445. if (!aot_compile_op_string_const(comp_ctx, func_ctx,
  1446. contents))
  1447. return false;
  1448. break;
  1449. }
  1450. case WASM_OP_STRING_MEASURE_UTF8:
  1451. case WASM_OP_STRING_MEASURE_WTF8:
  1452. case WASM_OP_STRING_MEASURE_WTF16:
  1453. {
  1454. EncodingFlag flag = WTF8;
  1455. if (opcode == WASM_OP_STRING_MEASURE_WTF16) {
  1456. flag = WTF16;
  1457. }
  1458. else if (opcode == WASM_OP_STRING_MEASURE_UTF8) {
  1459. flag = UTF8;
  1460. }
  1461. else if (opcode == WASM_OP_STRING_MEASURE_WTF8) {
  1462. flag = LOSSY_UTF8;
  1463. }
  1464. if (!aot_compile_op_string_measure(comp_ctx, func_ctx,
  1465. flag))
  1466. return false;
  1467. break;
  1468. }
  1469. case WASM_OP_STRING_ENCODE_UTF8:
  1470. case WASM_OP_STRING_ENCODE_WTF16:
  1471. case WASM_OP_STRING_ENCODE_LOSSY_UTF8:
  1472. case WASM_OP_STRING_ENCODE_WTF8:
  1473. {
  1474. EncodingFlag flag = WTF8;
  1475. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1476. bh_assert(mem_idx == 0);
  1477. if (opcode == WASM_OP_STRING_ENCODE_WTF16) {
  1478. flag = WTF16;
  1479. }
  1480. else if (opcode == WASM_OP_STRING_ENCODE_UTF8) {
  1481. flag = UTF8;
  1482. }
  1483. else if (opcode == WASM_OP_STRING_ENCODE_LOSSY_UTF8) {
  1484. flag = LOSSY_UTF8;
  1485. }
  1486. else if (opcode == WASM_OP_STRING_ENCODE_WTF8) {
  1487. flag = WTF8;
  1488. }
  1489. if (!aot_compile_op_string_encode(comp_ctx, func_ctx,
  1490. mem_idx, flag))
  1491. return false;
  1492. break;
  1493. }
  1494. case WASM_OP_STRING_CONCAT:
  1495. if (!aot_compile_op_string_concat(comp_ctx, func_ctx))
  1496. return false;
  1497. break;
  1498. case WASM_OP_STRING_EQ:
  1499. if (!aot_compile_op_string_eq(comp_ctx, func_ctx))
  1500. return false;
  1501. break;
  1502. case WASM_OP_STRING_IS_USV_SEQUENCE:
  1503. if (!aot_compile_op_string_is_usv_sequence(comp_ctx,
  1504. func_ctx))
  1505. return false;
  1506. break;
  1507. case WASM_OP_STRING_AS_WTF8:
  1508. if (!aot_compile_op_string_as_wtf8(comp_ctx, func_ctx))
  1509. return false;
  1510. break;
  1511. case WASM_OP_STRINGVIEW_WTF8_ADVANCE:
  1512. if (!aot_compile_op_stringview_wtf8_advance(comp_ctx,
  1513. func_ctx))
  1514. return false;
  1515. break;
  1516. case WASM_OP_STRINGVIEW_WTF8_ENCODE_UTF8:
  1517. case WASM_OP_STRINGVIEW_WTF8_ENCODE_LOSSY_UTF8:
  1518. case WASM_OP_STRINGVIEW_WTF8_ENCODE_WTF8:
  1519. {
  1520. EncodingFlag flag = WTF8;
  1521. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1522. bh_assert(mem_idx == 0);
  1523. if (opcode == WASM_OP_STRINGVIEW_WTF8_ENCODE_UTF8) {
  1524. flag = UTF8;
  1525. }
  1526. else if (opcode
  1527. == WASM_OP_STRINGVIEW_WTF8_ENCODE_LOSSY_UTF8) {
  1528. flag = LOSSY_UTF8;
  1529. }
  1530. else if (opcode
  1531. == WASM_OP_STRINGVIEW_WTF8_ENCODE_WTF8) {
  1532. flag = WTF8;
  1533. }
  1534. if (!aot_compile_op_stringview_wtf8_encode(
  1535. comp_ctx, func_ctx, mem_idx, flag))
  1536. return false;
  1537. break;
  1538. }
  1539. case WASM_OP_STRINGVIEW_WTF8_SLICE:
  1540. if (!aot_compile_op_stringview_wtf8_slice(comp_ctx,
  1541. func_ctx))
  1542. return false;
  1543. break;
  1544. case WASM_OP_STRING_AS_WTF16:
  1545. if (!aot_compile_op_string_as_wtf16(comp_ctx, func_ctx))
  1546. return false;
  1547. break;
  1548. case WASM_OP_STRINGVIEW_WTF16_LENGTH:
  1549. if (!aot_compile_op_stringview_wtf16_length(comp_ctx,
  1550. func_ctx))
  1551. return false;
  1552. break;
  1553. case WASM_OP_STRINGVIEW_WTF16_GET_CODEUNIT:
  1554. if (!aot_compile_op_stringview_wtf16_get_codeunit(
  1555. comp_ctx, func_ctx))
  1556. return false;
  1557. break;
  1558. case WASM_OP_STRINGVIEW_WTF16_ENCODE:
  1559. {
  1560. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1561. bh_assert(mem_idx == 0);
  1562. if (!aot_compile_op_stringview_wtf16_encode(
  1563. comp_ctx, func_ctx, mem_idx))
  1564. return false;
  1565. break;
  1566. }
  1567. case WASM_OP_STRINGVIEW_WTF16_SLICE:
  1568. if (!aot_compile_op_stringview_wtf16_slice(comp_ctx,
  1569. func_ctx))
  1570. return false;
  1571. break;
  1572. case WASM_OP_STRING_AS_ITER:
  1573. if (!aot_compile_op_string_as_iter(comp_ctx, func_ctx))
  1574. return false;
  1575. break;
  1576. case WASM_OP_STRINGVIEW_ITER_NEXT:
  1577. if (!aot_compile_op_stringview_iter_next(comp_ctx,
  1578. func_ctx))
  1579. return false;
  1580. break;
  1581. case WASM_OP_STRINGVIEW_ITER_ADVANCE:
  1582. if (!aot_compile_op_stringview_iter_advance(comp_ctx,
  1583. func_ctx))
  1584. return false;
  1585. break;
  1586. case WASM_OP_STRINGVIEW_ITER_REWIND:
  1587. if (!aot_compile_op_stringview_iter_rewind(comp_ctx,
  1588. func_ctx))
  1589. return false;
  1590. break;
  1591. case WASM_OP_STRINGVIEW_ITER_SLICE:
  1592. if (!aot_compile_op_stringview_iter_slice(comp_ctx,
  1593. func_ctx))
  1594. return false;
  1595. break;
  1596. case WASM_OP_STRING_NEW_UTF8_ARRAY:
  1597. case WASM_OP_STRING_NEW_WTF16_ARRAY:
  1598. case WASM_OP_STRING_NEW_LOSSY_UTF8_ARRAY:
  1599. case WASM_OP_STRING_NEW_WTF8_ARRAY:
  1600. {
  1601. EncodingFlag flag = WTF8;
  1602. if (opcode == WASM_OP_STRING_NEW_WTF16) {
  1603. flag = WTF16;
  1604. }
  1605. else if (opcode == WASM_OP_STRING_NEW_UTF8) {
  1606. flag = UTF8;
  1607. }
  1608. else if (opcode == WASM_OP_STRING_NEW_LOSSY_UTF8) {
  1609. flag = LOSSY_UTF8;
  1610. }
  1611. else if (opcode == WASM_OP_STRING_NEW_WTF8) {
  1612. flag = WTF8;
  1613. }
  1614. if (!aot_compile_op_string_new_array(comp_ctx, func_ctx,
  1615. flag))
  1616. return false;
  1617. break;
  1618. }
  1619. case WASM_OP_STRING_ENCODE_UTF8_ARRAY:
  1620. case WASM_OP_STRING_ENCODE_WTF16_ARRAY:
  1621. case WASM_OP_STRING_ENCODE_LOSSY_UTF8_ARRAY:
  1622. case WASM_OP_STRING_ENCODE_WTF8_ARRAY:
  1623. {
  1624. EncodingFlag flag = WTF8;
  1625. if (opcode == WASM_OP_STRING_ENCODE_WTF16) {
  1626. flag = WTF16;
  1627. }
  1628. else if (opcode == WASM_OP_STRING_ENCODE_UTF8) {
  1629. flag = UTF8;
  1630. }
  1631. else if (opcode == WASM_OP_STRING_ENCODE_LOSSY_UTF8) {
  1632. flag = LOSSY_UTF8;
  1633. }
  1634. else if (opcode == WASM_OP_STRING_ENCODE_WTF8) {
  1635. flag = WTF8;
  1636. }
  1637. if (!aot_compile_op_string_encode_array(comp_ctx,
  1638. func_ctx, flag))
  1639. return false;
  1640. break;
  1641. }
  1642. #endif /* end of WASM_ENABLE_STRINGREF != 0 */
  1643. default:
  1644. aot_set_last_error("unsupported opcode");
  1645. return false;
  1646. }
  1647. break;
  1648. }
  1649. #endif /* end of WASM_ENABLE_GC != 0 */
  1650. case WASM_OP_GET_LOCAL:
  1651. read_leb_uint32(frame_ip, frame_ip_end, local_idx);
  1652. if (!aot_compile_op_get_local(comp_ctx, func_ctx, local_idx))
  1653. return false;
  1654. break;
  1655. case WASM_OP_SET_LOCAL:
  1656. read_leb_uint32(frame_ip, frame_ip_end, local_idx);
  1657. if (!aot_compile_op_set_local(comp_ctx, func_ctx, local_idx))
  1658. return false;
  1659. break;
  1660. case WASM_OP_TEE_LOCAL:
  1661. read_leb_uint32(frame_ip, frame_ip_end, local_idx);
  1662. if (!aot_compile_op_tee_local(comp_ctx, func_ctx, local_idx))
  1663. return false;
  1664. break;
  1665. case WASM_OP_GET_GLOBAL:
  1666. case WASM_OP_GET_GLOBAL_64:
  1667. read_leb_uint32(frame_ip, frame_ip_end, global_idx);
  1668. if (!aot_compile_op_get_global(comp_ctx, func_ctx, global_idx))
  1669. return false;
  1670. break;
  1671. case WASM_OP_SET_GLOBAL:
  1672. case WASM_OP_SET_GLOBAL_64:
  1673. case WASM_OP_SET_GLOBAL_AUX_STACK:
  1674. read_leb_uint32(frame_ip, frame_ip_end, global_idx);
  1675. if (!aot_compile_op_set_global(
  1676. comp_ctx, func_ctx, global_idx,
  1677. opcode == WASM_OP_SET_GLOBAL_AUX_STACK ? true : false))
  1678. return false;
  1679. break;
  1680. case WASM_OP_I32_LOAD:
  1681. bytes = 4;
  1682. sign = true;
  1683. goto op_i32_load;
  1684. case WASM_OP_I32_LOAD8_S:
  1685. case WASM_OP_I32_LOAD8_U:
  1686. bytes = 1;
  1687. sign = (opcode == WASM_OP_I32_LOAD8_S) ? true : false;
  1688. goto op_i32_load;
  1689. case WASM_OP_I32_LOAD16_S:
  1690. case WASM_OP_I32_LOAD16_U:
  1691. bytes = 2;
  1692. sign = (opcode == WASM_OP_I32_LOAD16_S) ? true : false;
  1693. op_i32_load:
  1694. read_leb_uint32(frame_ip, frame_ip_end, align);
  1695. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1696. if (!aot_compile_op_i32_load(comp_ctx, func_ctx, align, offset,
  1697. bytes, sign, false))
  1698. return false;
  1699. break;
  1700. case WASM_OP_I64_LOAD:
  1701. bytes = 8;
  1702. sign = true;
  1703. goto op_i64_load;
  1704. case WASM_OP_I64_LOAD8_S:
  1705. case WASM_OP_I64_LOAD8_U:
  1706. bytes = 1;
  1707. sign = (opcode == WASM_OP_I64_LOAD8_S) ? true : false;
  1708. goto op_i64_load;
  1709. case WASM_OP_I64_LOAD16_S:
  1710. case WASM_OP_I64_LOAD16_U:
  1711. bytes = 2;
  1712. sign = (opcode == WASM_OP_I64_LOAD16_S) ? true : false;
  1713. goto op_i64_load;
  1714. case WASM_OP_I64_LOAD32_S:
  1715. case WASM_OP_I64_LOAD32_U:
  1716. bytes = 4;
  1717. sign = (opcode == WASM_OP_I64_LOAD32_S) ? true : false;
  1718. op_i64_load:
  1719. read_leb_uint32(frame_ip, frame_ip_end, align);
  1720. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1721. if (!aot_compile_op_i64_load(comp_ctx, func_ctx, align, offset,
  1722. bytes, sign, false))
  1723. return false;
  1724. break;
  1725. case WASM_OP_F32_LOAD:
  1726. read_leb_uint32(frame_ip, frame_ip_end, align);
  1727. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1728. if (!aot_compile_op_f32_load(comp_ctx, func_ctx, align, offset))
  1729. return false;
  1730. break;
  1731. case WASM_OP_F64_LOAD:
  1732. read_leb_uint32(frame_ip, frame_ip_end, align);
  1733. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1734. if (!aot_compile_op_f64_load(comp_ctx, func_ctx, align, offset))
  1735. return false;
  1736. break;
  1737. case WASM_OP_I32_STORE:
  1738. bytes = 4;
  1739. goto op_i32_store;
  1740. case WASM_OP_I32_STORE8:
  1741. bytes = 1;
  1742. goto op_i32_store;
  1743. case WASM_OP_I32_STORE16:
  1744. bytes = 2;
  1745. op_i32_store:
  1746. read_leb_uint32(frame_ip, frame_ip_end, align);
  1747. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1748. if (!aot_compile_op_i32_store(comp_ctx, func_ctx, align, offset,
  1749. bytes, false))
  1750. return false;
  1751. break;
  1752. case WASM_OP_I64_STORE:
  1753. bytes = 8;
  1754. goto op_i64_store;
  1755. case WASM_OP_I64_STORE8:
  1756. bytes = 1;
  1757. goto op_i64_store;
  1758. case WASM_OP_I64_STORE16:
  1759. bytes = 2;
  1760. goto op_i64_store;
  1761. case WASM_OP_I64_STORE32:
  1762. bytes = 4;
  1763. op_i64_store:
  1764. read_leb_uint32(frame_ip, frame_ip_end, align);
  1765. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1766. if (!aot_compile_op_i64_store(comp_ctx, func_ctx, align, offset,
  1767. bytes, false))
  1768. return false;
  1769. break;
  1770. case WASM_OP_F32_STORE:
  1771. read_leb_uint32(frame_ip, frame_ip_end, align);
  1772. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1773. if (!aot_compile_op_f32_store(comp_ctx, func_ctx, align,
  1774. offset))
  1775. return false;
  1776. break;
  1777. case WASM_OP_F64_STORE:
  1778. read_leb_uint32(frame_ip, frame_ip_end, align);
  1779. read_leb_uint32(frame_ip, frame_ip_end, offset);
  1780. if (!aot_compile_op_f64_store(comp_ctx, func_ctx, align,
  1781. offset))
  1782. return false;
  1783. break;
  1784. case WASM_OP_MEMORY_SIZE:
  1785. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1786. if (!aot_compile_op_memory_size(comp_ctx, func_ctx))
  1787. return false;
  1788. (void)mem_idx;
  1789. break;
  1790. case WASM_OP_MEMORY_GROW:
  1791. read_leb_uint32(frame_ip, frame_ip_end, mem_idx);
  1792. if (!aot_compile_op_memory_grow(comp_ctx, func_ctx))
  1793. return false;
  1794. break;
  1795. case WASM_OP_I32_CONST:
  1796. read_leb_int32(frame_ip, frame_ip_end, i32_const);
  1797. if (!aot_compile_op_i32_const(comp_ctx, func_ctx, i32_const))
  1798. return false;
  1799. break;
  1800. case WASM_OP_I64_CONST:
  1801. read_leb_int64(frame_ip, frame_ip_end, i64_const);
  1802. if (!aot_compile_op_i64_const(comp_ctx, func_ctx, i64_const))
  1803. return false;
  1804. break;
  1805. case WASM_OP_F32_CONST:
  1806. p_f32 = (uint8 *)&f32_const;
  1807. for (i = 0; i < sizeof(float32); i++)
  1808. *p_f32++ = *frame_ip++;
  1809. if (!aot_compile_op_f32_const(comp_ctx, func_ctx, f32_const))
  1810. return false;
  1811. break;
  1812. case WASM_OP_F64_CONST:
  1813. p_f64 = (uint8 *)&f64_const;
  1814. for (i = 0; i < sizeof(float64); i++)
  1815. *p_f64++ = *frame_ip++;
  1816. if (!aot_compile_op_f64_const(comp_ctx, func_ctx, f64_const))
  1817. return false;
  1818. break;
  1819. case WASM_OP_I32_EQZ:
  1820. case WASM_OP_I32_EQ:
  1821. case WASM_OP_I32_NE:
  1822. case WASM_OP_I32_LT_S:
  1823. case WASM_OP_I32_LT_U:
  1824. case WASM_OP_I32_GT_S:
  1825. case WASM_OP_I32_GT_U:
  1826. case WASM_OP_I32_LE_S:
  1827. case WASM_OP_I32_LE_U:
  1828. case WASM_OP_I32_GE_S:
  1829. case WASM_OP_I32_GE_U:
  1830. if (!aot_compile_op_i32_compare(
  1831. comp_ctx, func_ctx, INT_EQZ + opcode - WASM_OP_I32_EQZ))
  1832. return false;
  1833. break;
  1834. case WASM_OP_I64_EQZ:
  1835. case WASM_OP_I64_EQ:
  1836. case WASM_OP_I64_NE:
  1837. case WASM_OP_I64_LT_S:
  1838. case WASM_OP_I64_LT_U:
  1839. case WASM_OP_I64_GT_S:
  1840. case WASM_OP_I64_GT_U:
  1841. case WASM_OP_I64_LE_S:
  1842. case WASM_OP_I64_LE_U:
  1843. case WASM_OP_I64_GE_S:
  1844. case WASM_OP_I64_GE_U:
  1845. if (!aot_compile_op_i64_compare(
  1846. comp_ctx, func_ctx, INT_EQZ + opcode - WASM_OP_I64_EQZ))
  1847. return false;
  1848. break;
  1849. case WASM_OP_F32_EQ:
  1850. case WASM_OP_F32_NE:
  1851. case WASM_OP_F32_LT:
  1852. case WASM_OP_F32_GT:
  1853. case WASM_OP_F32_LE:
  1854. case WASM_OP_F32_GE:
  1855. if (!aot_compile_op_f32_compare(
  1856. comp_ctx, func_ctx, FLOAT_EQ + opcode - WASM_OP_F32_EQ))
  1857. return false;
  1858. break;
  1859. case WASM_OP_F64_EQ:
  1860. case WASM_OP_F64_NE:
  1861. case WASM_OP_F64_LT:
  1862. case WASM_OP_F64_GT:
  1863. case WASM_OP_F64_LE:
  1864. case WASM_OP_F64_GE:
  1865. if (!aot_compile_op_f64_compare(
  1866. comp_ctx, func_ctx, FLOAT_EQ + opcode - WASM_OP_F64_EQ))
  1867. return false;
  1868. break;
  1869. case WASM_OP_I32_CLZ:
  1870. if (!aot_compile_op_i32_clz(comp_ctx, func_ctx))
  1871. return false;
  1872. break;
  1873. case WASM_OP_I32_CTZ:
  1874. if (!aot_compile_op_i32_ctz(comp_ctx, func_ctx))
  1875. return false;
  1876. break;
  1877. case WASM_OP_I32_POPCNT:
  1878. if (!aot_compile_op_i32_popcnt(comp_ctx, func_ctx))
  1879. return false;
  1880. break;
  1881. case WASM_OP_I32_ADD:
  1882. case WASM_OP_I32_SUB:
  1883. case WASM_OP_I32_MUL:
  1884. case WASM_OP_I32_DIV_S:
  1885. case WASM_OP_I32_DIV_U:
  1886. case WASM_OP_I32_REM_S:
  1887. case WASM_OP_I32_REM_U:
  1888. if (!aot_compile_op_i32_arithmetic(
  1889. comp_ctx, func_ctx, INT_ADD + opcode - WASM_OP_I32_ADD,
  1890. &frame_ip))
  1891. return false;
  1892. break;
  1893. case WASM_OP_I32_AND:
  1894. case WASM_OP_I32_OR:
  1895. case WASM_OP_I32_XOR:
  1896. if (!aot_compile_op_i32_bitwise(
  1897. comp_ctx, func_ctx, INT_SHL + opcode - WASM_OP_I32_AND))
  1898. return false;
  1899. break;
  1900. case WASM_OP_I32_SHL:
  1901. case WASM_OP_I32_SHR_S:
  1902. case WASM_OP_I32_SHR_U:
  1903. case WASM_OP_I32_ROTL:
  1904. case WASM_OP_I32_ROTR:
  1905. if (!aot_compile_op_i32_shift(
  1906. comp_ctx, func_ctx, INT_SHL + opcode - WASM_OP_I32_SHL))
  1907. return false;
  1908. break;
  1909. case WASM_OP_I64_CLZ:
  1910. if (!aot_compile_op_i64_clz(comp_ctx, func_ctx))
  1911. return false;
  1912. break;
  1913. case WASM_OP_I64_CTZ:
  1914. if (!aot_compile_op_i64_ctz(comp_ctx, func_ctx))
  1915. return false;
  1916. break;
  1917. case WASM_OP_I64_POPCNT:
  1918. if (!aot_compile_op_i64_popcnt(comp_ctx, func_ctx))
  1919. return false;
  1920. break;
  1921. case WASM_OP_I64_ADD:
  1922. case WASM_OP_I64_SUB:
  1923. case WASM_OP_I64_MUL:
  1924. case WASM_OP_I64_DIV_S:
  1925. case WASM_OP_I64_DIV_U:
  1926. case WASM_OP_I64_REM_S:
  1927. case WASM_OP_I64_REM_U:
  1928. if (!aot_compile_op_i64_arithmetic(
  1929. comp_ctx, func_ctx, INT_ADD + opcode - WASM_OP_I64_ADD,
  1930. &frame_ip))
  1931. return false;
  1932. break;
  1933. case WASM_OP_I64_AND:
  1934. case WASM_OP_I64_OR:
  1935. case WASM_OP_I64_XOR:
  1936. if (!aot_compile_op_i64_bitwise(
  1937. comp_ctx, func_ctx, INT_SHL + opcode - WASM_OP_I64_AND))
  1938. return false;
  1939. break;
  1940. case WASM_OP_I64_SHL:
  1941. case WASM_OP_I64_SHR_S:
  1942. case WASM_OP_I64_SHR_U:
  1943. case WASM_OP_I64_ROTL:
  1944. case WASM_OP_I64_ROTR:
  1945. if (!aot_compile_op_i64_shift(
  1946. comp_ctx, func_ctx, INT_SHL + opcode - WASM_OP_I64_SHL))
  1947. return false;
  1948. break;
  1949. case WASM_OP_F32_ABS:
  1950. case WASM_OP_F32_NEG:
  1951. case WASM_OP_F32_CEIL:
  1952. case WASM_OP_F32_FLOOR:
  1953. case WASM_OP_F32_TRUNC:
  1954. case WASM_OP_F32_NEAREST:
  1955. case WASM_OP_F32_SQRT:
  1956. if (!aot_compile_op_f32_math(comp_ctx, func_ctx,
  1957. FLOAT_ABS + opcode
  1958. - WASM_OP_F32_ABS))
  1959. return false;
  1960. break;
  1961. case WASM_OP_F32_ADD:
  1962. case WASM_OP_F32_SUB:
  1963. case WASM_OP_F32_MUL:
  1964. case WASM_OP_F32_DIV:
  1965. case WASM_OP_F32_MIN:
  1966. case WASM_OP_F32_MAX:
  1967. if (!aot_compile_op_f32_arithmetic(comp_ctx, func_ctx,
  1968. FLOAT_ADD + opcode
  1969. - WASM_OP_F32_ADD))
  1970. return false;
  1971. break;
  1972. case WASM_OP_F32_COPYSIGN:
  1973. if (!aot_compile_op_f32_copysign(comp_ctx, func_ctx))
  1974. return false;
  1975. break;
  1976. case WASM_OP_F64_ABS:
  1977. case WASM_OP_F64_NEG:
  1978. case WASM_OP_F64_CEIL:
  1979. case WASM_OP_F64_FLOOR:
  1980. case WASM_OP_F64_TRUNC:
  1981. case WASM_OP_F64_NEAREST:
  1982. case WASM_OP_F64_SQRT:
  1983. if (!aot_compile_op_f64_math(comp_ctx, func_ctx,
  1984. FLOAT_ABS + opcode
  1985. - WASM_OP_F64_ABS))
  1986. return false;
  1987. break;
  1988. case WASM_OP_F64_ADD:
  1989. case WASM_OP_F64_SUB:
  1990. case WASM_OP_F64_MUL:
  1991. case WASM_OP_F64_DIV:
  1992. case WASM_OP_F64_MIN:
  1993. case WASM_OP_F64_MAX:
  1994. if (!aot_compile_op_f64_arithmetic(comp_ctx, func_ctx,
  1995. FLOAT_ADD + opcode
  1996. - WASM_OP_F64_ADD))
  1997. return false;
  1998. break;
  1999. case WASM_OP_F64_COPYSIGN:
  2000. if (!aot_compile_op_f64_copysign(comp_ctx, func_ctx))
  2001. return false;
  2002. break;
  2003. case WASM_OP_I32_WRAP_I64:
  2004. if (!aot_compile_op_i32_wrap_i64(comp_ctx, func_ctx))
  2005. return false;
  2006. break;
  2007. case WASM_OP_I32_TRUNC_S_F32:
  2008. case WASM_OP_I32_TRUNC_U_F32:
  2009. sign = (opcode == WASM_OP_I32_TRUNC_S_F32) ? true : false;
  2010. if (!aot_compile_op_i32_trunc_f32(comp_ctx, func_ctx, sign,
  2011. false))
  2012. return false;
  2013. break;
  2014. case WASM_OP_I32_TRUNC_S_F64:
  2015. case WASM_OP_I32_TRUNC_U_F64:
  2016. sign = (opcode == WASM_OP_I32_TRUNC_S_F64) ? true : false;
  2017. if (!aot_compile_op_i32_trunc_f64(comp_ctx, func_ctx, sign,
  2018. false))
  2019. return false;
  2020. break;
  2021. case WASM_OP_I64_EXTEND_S_I32:
  2022. case WASM_OP_I64_EXTEND_U_I32:
  2023. sign = (opcode == WASM_OP_I64_EXTEND_S_I32) ? true : false;
  2024. if (!aot_compile_op_i64_extend_i32(comp_ctx, func_ctx, sign))
  2025. return false;
  2026. break;
  2027. case WASM_OP_I64_TRUNC_S_F32:
  2028. case WASM_OP_I64_TRUNC_U_F32:
  2029. sign = (opcode == WASM_OP_I64_TRUNC_S_F32) ? true : false;
  2030. if (!aot_compile_op_i64_trunc_f32(comp_ctx, func_ctx, sign,
  2031. false))
  2032. return false;
  2033. break;
  2034. case WASM_OP_I64_TRUNC_S_F64:
  2035. case WASM_OP_I64_TRUNC_U_F64:
  2036. sign = (opcode == WASM_OP_I64_TRUNC_S_F64) ? true : false;
  2037. if (!aot_compile_op_i64_trunc_f64(comp_ctx, func_ctx, sign,
  2038. false))
  2039. return false;
  2040. break;
  2041. case WASM_OP_F32_CONVERT_S_I32:
  2042. case WASM_OP_F32_CONVERT_U_I32:
  2043. sign = (opcode == WASM_OP_F32_CONVERT_S_I32) ? true : false;
  2044. if (!aot_compile_op_f32_convert_i32(comp_ctx, func_ctx, sign))
  2045. return false;
  2046. break;
  2047. case WASM_OP_F32_CONVERT_S_I64:
  2048. case WASM_OP_F32_CONVERT_U_I64:
  2049. sign = (opcode == WASM_OP_F32_CONVERT_S_I64) ? true : false;
  2050. if (!aot_compile_op_f32_convert_i64(comp_ctx, func_ctx, sign))
  2051. return false;
  2052. break;
  2053. case WASM_OP_F32_DEMOTE_F64:
  2054. if (!aot_compile_op_f32_demote_f64(comp_ctx, func_ctx))
  2055. return false;
  2056. break;
  2057. case WASM_OP_F64_CONVERT_S_I32:
  2058. case WASM_OP_F64_CONVERT_U_I32:
  2059. sign = (opcode == WASM_OP_F64_CONVERT_S_I32) ? true : false;
  2060. if (!aot_compile_op_f64_convert_i32(comp_ctx, func_ctx, sign))
  2061. return false;
  2062. break;
  2063. case WASM_OP_F64_CONVERT_S_I64:
  2064. case WASM_OP_F64_CONVERT_U_I64:
  2065. sign = (opcode == WASM_OP_F64_CONVERT_S_I64) ? true : false;
  2066. if (!aot_compile_op_f64_convert_i64(comp_ctx, func_ctx, sign))
  2067. return false;
  2068. break;
  2069. case WASM_OP_F64_PROMOTE_F32:
  2070. if (!aot_compile_op_f64_promote_f32(comp_ctx, func_ctx))
  2071. return false;
  2072. break;
  2073. case WASM_OP_I32_REINTERPRET_F32:
  2074. if (!aot_compile_op_i32_reinterpret_f32(comp_ctx, func_ctx))
  2075. return false;
  2076. break;
  2077. case WASM_OP_I64_REINTERPRET_F64:
  2078. if (!aot_compile_op_i64_reinterpret_f64(comp_ctx, func_ctx))
  2079. return false;
  2080. break;
  2081. case WASM_OP_F32_REINTERPRET_I32:
  2082. if (!aot_compile_op_f32_reinterpret_i32(comp_ctx, func_ctx))
  2083. return false;
  2084. break;
  2085. case WASM_OP_F64_REINTERPRET_I64:
  2086. if (!aot_compile_op_f64_reinterpret_i64(comp_ctx, func_ctx))
  2087. return false;
  2088. break;
  2089. case WASM_OP_I32_EXTEND8_S:
  2090. if (!aot_compile_op_i32_extend_i32(comp_ctx, func_ctx, 8))
  2091. return false;
  2092. break;
  2093. case WASM_OP_I32_EXTEND16_S:
  2094. if (!aot_compile_op_i32_extend_i32(comp_ctx, func_ctx, 16))
  2095. return false;
  2096. break;
  2097. case WASM_OP_I64_EXTEND8_S:
  2098. if (!aot_compile_op_i64_extend_i64(comp_ctx, func_ctx, 8))
  2099. return false;
  2100. break;
  2101. case WASM_OP_I64_EXTEND16_S:
  2102. if (!aot_compile_op_i64_extend_i64(comp_ctx, func_ctx, 16))
  2103. return false;
  2104. break;
  2105. case WASM_OP_I64_EXTEND32_S:
  2106. if (!aot_compile_op_i64_extend_i64(comp_ctx, func_ctx, 32))
  2107. return false;
  2108. break;
  2109. case WASM_OP_MISC_PREFIX:
  2110. {
  2111. uint32 opcode1;
  2112. read_leb_uint32(frame_ip, frame_ip_end, opcode1);
  2113. /* opcode1 was checked in loader and is no larger than
  2114. UINT8_MAX */
  2115. opcode = (uint8)opcode1;
  2116. #if WASM_ENABLE_BULK_MEMORY != 0
  2117. if (WASM_OP_MEMORY_INIT <= opcode
  2118. && opcode <= WASM_OP_MEMORY_FILL
  2119. && !comp_ctx->enable_bulk_memory) {
  2120. goto unsupport_bulk_memory;
  2121. }
  2122. #endif
  2123. #if WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0
  2124. if (WASM_OP_TABLE_INIT <= opcode && opcode <= WASM_OP_TABLE_FILL
  2125. && (!comp_ctx->enable_ref_types && !comp_ctx->enable_gc)) {
  2126. goto unsupport_ref_types;
  2127. }
  2128. #endif
  2129. switch (opcode) {
  2130. case WASM_OP_I32_TRUNC_SAT_S_F32:
  2131. case WASM_OP_I32_TRUNC_SAT_U_F32:
  2132. sign = (opcode == WASM_OP_I32_TRUNC_SAT_S_F32) ? true
  2133. : false;
  2134. if (!aot_compile_op_i32_trunc_f32(comp_ctx, func_ctx,
  2135. sign, true))
  2136. return false;
  2137. break;
  2138. case WASM_OP_I32_TRUNC_SAT_S_F64:
  2139. case WASM_OP_I32_TRUNC_SAT_U_F64:
  2140. sign = (opcode == WASM_OP_I32_TRUNC_SAT_S_F64) ? true
  2141. : false;
  2142. if (!aot_compile_op_i32_trunc_f64(comp_ctx, func_ctx,
  2143. sign, true))
  2144. return false;
  2145. break;
  2146. case WASM_OP_I64_TRUNC_SAT_S_F32:
  2147. case WASM_OP_I64_TRUNC_SAT_U_F32:
  2148. sign = (opcode == WASM_OP_I64_TRUNC_SAT_S_F32) ? true
  2149. : false;
  2150. if (!aot_compile_op_i64_trunc_f32(comp_ctx, func_ctx,
  2151. sign, true))
  2152. return false;
  2153. break;
  2154. case WASM_OP_I64_TRUNC_SAT_S_F64:
  2155. case WASM_OP_I64_TRUNC_SAT_U_F64:
  2156. sign = (opcode == WASM_OP_I64_TRUNC_SAT_S_F64) ? true
  2157. : false;
  2158. if (!aot_compile_op_i64_trunc_f64(comp_ctx, func_ctx,
  2159. sign, true))
  2160. return false;
  2161. break;
  2162. #if WASM_ENABLE_BULK_MEMORY != 0
  2163. case WASM_OP_MEMORY_INIT:
  2164. {
  2165. uint32 seg_index;
  2166. read_leb_uint32(frame_ip, frame_ip_end, seg_index);
  2167. frame_ip++;
  2168. if (!aot_compile_op_memory_init(comp_ctx, func_ctx,
  2169. seg_index))
  2170. return false;
  2171. break;
  2172. }
  2173. case WASM_OP_DATA_DROP:
  2174. {
  2175. uint32 seg_index;
  2176. read_leb_uint32(frame_ip, frame_ip_end, seg_index);
  2177. if (!aot_compile_op_data_drop(comp_ctx, func_ctx,
  2178. seg_index))
  2179. return false;
  2180. break;
  2181. }
  2182. case WASM_OP_MEMORY_COPY:
  2183. {
  2184. frame_ip += 2;
  2185. if (!aot_compile_op_memory_copy(comp_ctx, func_ctx))
  2186. return false;
  2187. break;
  2188. }
  2189. case WASM_OP_MEMORY_FILL:
  2190. {
  2191. frame_ip++;
  2192. if (!aot_compile_op_memory_fill(comp_ctx, func_ctx))
  2193. return false;
  2194. break;
  2195. }
  2196. #endif /* WASM_ENABLE_BULK_MEMORY */
  2197. #if WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0
  2198. case WASM_OP_TABLE_INIT:
  2199. {
  2200. uint32 tbl_idx, tbl_seg_idx;
  2201. read_leb_uint32(frame_ip, frame_ip_end, tbl_seg_idx);
  2202. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  2203. if (!aot_compile_op_table_init(comp_ctx, func_ctx,
  2204. tbl_idx, tbl_seg_idx))
  2205. return false;
  2206. break;
  2207. }
  2208. case WASM_OP_ELEM_DROP:
  2209. {
  2210. uint32 tbl_seg_idx;
  2211. read_leb_uint32(frame_ip, frame_ip_end, tbl_seg_idx);
  2212. if (!aot_compile_op_elem_drop(comp_ctx, func_ctx,
  2213. tbl_seg_idx))
  2214. return false;
  2215. break;
  2216. }
  2217. case WASM_OP_TABLE_COPY:
  2218. {
  2219. uint32 src_tbl_idx, dst_tbl_idx;
  2220. read_leb_uint32(frame_ip, frame_ip_end, dst_tbl_idx);
  2221. read_leb_uint32(frame_ip, frame_ip_end, src_tbl_idx);
  2222. if (!aot_compile_op_table_copy(
  2223. comp_ctx, func_ctx, src_tbl_idx, dst_tbl_idx))
  2224. return false;
  2225. break;
  2226. }
  2227. case WASM_OP_TABLE_GROW:
  2228. {
  2229. uint32 tbl_idx;
  2230. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  2231. if (!aot_compile_op_table_grow(comp_ctx, func_ctx,
  2232. tbl_idx))
  2233. return false;
  2234. break;
  2235. }
  2236. case WASM_OP_TABLE_SIZE:
  2237. {
  2238. uint32 tbl_idx;
  2239. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  2240. if (!aot_compile_op_table_size(comp_ctx, func_ctx,
  2241. tbl_idx))
  2242. return false;
  2243. break;
  2244. }
  2245. case WASM_OP_TABLE_FILL:
  2246. {
  2247. uint32 tbl_idx;
  2248. read_leb_uint32(frame_ip, frame_ip_end, tbl_idx);
  2249. if (!aot_compile_op_table_fill(comp_ctx, func_ctx,
  2250. tbl_idx))
  2251. return false;
  2252. break;
  2253. }
  2254. #endif /* WASM_ENABLE_REF_TYPES || WASM_ENABLE_GC */
  2255. default:
  2256. aot_set_last_error("unsupported opcode");
  2257. return false;
  2258. }
  2259. break;
  2260. }
  2261. #if WASM_ENABLE_SHARED_MEMORY != 0
  2262. case WASM_OP_ATOMIC_PREFIX:
  2263. {
  2264. uint8 bin_op, op_type;
  2265. uint32 opcode1;
  2266. read_leb_uint32(frame_ip, frame_ip_end, opcode1);
  2267. /* opcode1 was checked in loader and is no larger than
  2268. UINT8_MAX */
  2269. opcode = (uint8)opcode1;
  2270. if (opcode != WASM_OP_ATOMIC_FENCE) {
  2271. read_leb_uint32(frame_ip, frame_ip_end, align);
  2272. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2273. }
  2274. switch (opcode) {
  2275. case WASM_OP_ATOMIC_WAIT32:
  2276. if (!aot_compile_op_atomic_wait(comp_ctx, func_ctx,
  2277. VALUE_TYPE_I32, align,
  2278. offset, 4))
  2279. return false;
  2280. break;
  2281. case WASM_OP_ATOMIC_WAIT64:
  2282. if (!aot_compile_op_atomic_wait(comp_ctx, func_ctx,
  2283. VALUE_TYPE_I64, align,
  2284. offset, 8))
  2285. return false;
  2286. break;
  2287. case WASM_OP_ATOMIC_NOTIFY:
  2288. if (!aot_compiler_op_atomic_notify(
  2289. comp_ctx, func_ctx, align, offset, bytes))
  2290. return false;
  2291. break;
  2292. case WASM_OP_ATOMIC_FENCE:
  2293. /* Skip memory index */
  2294. frame_ip++;
  2295. if (!aot_compiler_op_atomic_fence(comp_ctx, func_ctx))
  2296. return false;
  2297. break;
  2298. case WASM_OP_ATOMIC_I32_LOAD:
  2299. bytes = 4;
  2300. goto op_atomic_i32_load;
  2301. case WASM_OP_ATOMIC_I32_LOAD8_U:
  2302. bytes = 1;
  2303. goto op_atomic_i32_load;
  2304. case WASM_OP_ATOMIC_I32_LOAD16_U:
  2305. bytes = 2;
  2306. op_atomic_i32_load:
  2307. if (!aot_compile_op_i32_load(comp_ctx, func_ctx, align,
  2308. offset, bytes, sign, true))
  2309. return false;
  2310. break;
  2311. case WASM_OP_ATOMIC_I64_LOAD:
  2312. bytes = 8;
  2313. goto op_atomic_i64_load;
  2314. case WASM_OP_ATOMIC_I64_LOAD8_U:
  2315. bytes = 1;
  2316. goto op_atomic_i64_load;
  2317. case WASM_OP_ATOMIC_I64_LOAD16_U:
  2318. bytes = 2;
  2319. goto op_atomic_i64_load;
  2320. case WASM_OP_ATOMIC_I64_LOAD32_U:
  2321. bytes = 4;
  2322. op_atomic_i64_load:
  2323. if (!aot_compile_op_i64_load(comp_ctx, func_ctx, align,
  2324. offset, bytes, sign, true))
  2325. return false;
  2326. break;
  2327. case WASM_OP_ATOMIC_I32_STORE:
  2328. bytes = 4;
  2329. goto op_atomic_i32_store;
  2330. case WASM_OP_ATOMIC_I32_STORE8:
  2331. bytes = 1;
  2332. goto op_atomic_i32_store;
  2333. case WASM_OP_ATOMIC_I32_STORE16:
  2334. bytes = 2;
  2335. op_atomic_i32_store:
  2336. if (!aot_compile_op_i32_store(comp_ctx, func_ctx, align,
  2337. offset, bytes, true))
  2338. return false;
  2339. break;
  2340. case WASM_OP_ATOMIC_I64_STORE:
  2341. bytes = 8;
  2342. goto op_atomic_i64_store;
  2343. case WASM_OP_ATOMIC_I64_STORE8:
  2344. bytes = 1;
  2345. goto op_atomic_i64_store;
  2346. case WASM_OP_ATOMIC_I64_STORE16:
  2347. bytes = 2;
  2348. goto op_atomic_i64_store;
  2349. case WASM_OP_ATOMIC_I64_STORE32:
  2350. bytes = 4;
  2351. op_atomic_i64_store:
  2352. if (!aot_compile_op_i64_store(comp_ctx, func_ctx, align,
  2353. offset, bytes, true))
  2354. return false;
  2355. break;
  2356. case WASM_OP_ATOMIC_RMW_I32_CMPXCHG:
  2357. bytes = 4;
  2358. op_type = VALUE_TYPE_I32;
  2359. goto op_atomic_cmpxchg;
  2360. case WASM_OP_ATOMIC_RMW_I64_CMPXCHG:
  2361. bytes = 8;
  2362. op_type = VALUE_TYPE_I64;
  2363. goto op_atomic_cmpxchg;
  2364. case WASM_OP_ATOMIC_RMW_I32_CMPXCHG8_U:
  2365. bytes = 1;
  2366. op_type = VALUE_TYPE_I32;
  2367. goto op_atomic_cmpxchg;
  2368. case WASM_OP_ATOMIC_RMW_I32_CMPXCHG16_U:
  2369. bytes = 2;
  2370. op_type = VALUE_TYPE_I32;
  2371. goto op_atomic_cmpxchg;
  2372. case WASM_OP_ATOMIC_RMW_I64_CMPXCHG8_U:
  2373. bytes = 1;
  2374. op_type = VALUE_TYPE_I64;
  2375. goto op_atomic_cmpxchg;
  2376. case WASM_OP_ATOMIC_RMW_I64_CMPXCHG16_U:
  2377. bytes = 2;
  2378. op_type = VALUE_TYPE_I64;
  2379. goto op_atomic_cmpxchg;
  2380. case WASM_OP_ATOMIC_RMW_I64_CMPXCHG32_U:
  2381. bytes = 4;
  2382. op_type = VALUE_TYPE_I64;
  2383. op_atomic_cmpxchg:
  2384. if (!aot_compile_op_atomic_cmpxchg(comp_ctx, func_ctx,
  2385. op_type, align,
  2386. offset, bytes))
  2387. return false;
  2388. break;
  2389. COMPILE_ATOMIC_RMW(Add, ADD);
  2390. COMPILE_ATOMIC_RMW(Sub, SUB);
  2391. COMPILE_ATOMIC_RMW(And, AND);
  2392. COMPILE_ATOMIC_RMW(Or, OR);
  2393. COMPILE_ATOMIC_RMW(Xor, XOR);
  2394. COMPILE_ATOMIC_RMW(Xchg, XCHG);
  2395. build_atomic_rmw:
  2396. if (!aot_compile_op_atomic_rmw(comp_ctx, func_ctx,
  2397. bin_op, op_type, align,
  2398. offset, bytes))
  2399. return false;
  2400. break;
  2401. default:
  2402. aot_set_last_error("unsupported opcode");
  2403. return false;
  2404. }
  2405. break;
  2406. }
  2407. #endif /* end of WASM_ENABLE_SHARED_MEMORY */
  2408. #if WASM_ENABLE_SIMD != 0
  2409. case WASM_OP_SIMD_PREFIX:
  2410. {
  2411. uint32 opcode1;
  2412. if (!comp_ctx->enable_simd) {
  2413. goto unsupport_simd;
  2414. }
  2415. read_leb_uint32(frame_ip, frame_ip_end, opcode1);
  2416. /* opcode1 was checked in loader and is no larger than
  2417. UINT8_MAX */
  2418. opcode = (uint8)opcode1;
  2419. /* follow the order of enum WASMSimdEXTOpcode in
  2420. wasm_opcode.h */
  2421. switch (opcode) {
  2422. /* Memory instruction */
  2423. case SIMD_v128_load:
  2424. {
  2425. read_leb_uint32(frame_ip, frame_ip_end, align);
  2426. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2427. if (!aot_compile_simd_v128_load(comp_ctx, func_ctx,
  2428. align, offset))
  2429. return false;
  2430. break;
  2431. }
  2432. case SIMD_v128_load8x8_s:
  2433. case SIMD_v128_load8x8_u:
  2434. case SIMD_v128_load16x4_s:
  2435. case SIMD_v128_load16x4_u:
  2436. case SIMD_v128_load32x2_s:
  2437. case SIMD_v128_load32x2_u:
  2438. {
  2439. read_leb_uint32(frame_ip, frame_ip_end, align);
  2440. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2441. if (!aot_compile_simd_load_extend(
  2442. comp_ctx, func_ctx, opcode, align, offset))
  2443. return false;
  2444. break;
  2445. }
  2446. case SIMD_v128_load8_splat:
  2447. case SIMD_v128_load16_splat:
  2448. case SIMD_v128_load32_splat:
  2449. case SIMD_v128_load64_splat:
  2450. {
  2451. read_leb_uint32(frame_ip, frame_ip_end, align);
  2452. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2453. if (!aot_compile_simd_load_splat(comp_ctx, func_ctx,
  2454. opcode, align, offset))
  2455. return false;
  2456. break;
  2457. }
  2458. case SIMD_v128_store:
  2459. {
  2460. read_leb_uint32(frame_ip, frame_ip_end, align);
  2461. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2462. if (!aot_compile_simd_v128_store(comp_ctx, func_ctx,
  2463. align, offset))
  2464. return false;
  2465. break;
  2466. }
  2467. /* Basic operation */
  2468. case SIMD_v128_const:
  2469. {
  2470. if (!aot_compile_simd_v128_const(comp_ctx, func_ctx,
  2471. frame_ip))
  2472. return false;
  2473. frame_ip += 16;
  2474. break;
  2475. }
  2476. case SIMD_v8x16_shuffle:
  2477. {
  2478. if (!aot_compile_simd_shuffle(comp_ctx, func_ctx,
  2479. frame_ip))
  2480. return false;
  2481. frame_ip += 16;
  2482. break;
  2483. }
  2484. case SIMD_v8x16_swizzle:
  2485. {
  2486. if (!aot_compile_simd_swizzle(comp_ctx, func_ctx))
  2487. return false;
  2488. break;
  2489. }
  2490. /* Splat operation */
  2491. case SIMD_i8x16_splat:
  2492. case SIMD_i16x8_splat:
  2493. case SIMD_i32x4_splat:
  2494. case SIMD_i64x2_splat:
  2495. case SIMD_f32x4_splat:
  2496. case SIMD_f64x2_splat:
  2497. {
  2498. if (!aot_compile_simd_splat(comp_ctx, func_ctx, opcode))
  2499. return false;
  2500. break;
  2501. }
  2502. /* Lane operation */
  2503. case SIMD_i8x16_extract_lane_s:
  2504. case SIMD_i8x16_extract_lane_u:
  2505. {
  2506. if (!aot_compile_simd_extract_i8x16(
  2507. comp_ctx, func_ctx, *frame_ip++,
  2508. SIMD_i8x16_extract_lane_s == opcode))
  2509. return false;
  2510. break;
  2511. }
  2512. case SIMD_i8x16_replace_lane:
  2513. {
  2514. if (!aot_compile_simd_replace_i8x16(comp_ctx, func_ctx,
  2515. *frame_ip++))
  2516. return false;
  2517. break;
  2518. }
  2519. case SIMD_i16x8_extract_lane_s:
  2520. case SIMD_i16x8_extract_lane_u:
  2521. {
  2522. if (!aot_compile_simd_extract_i16x8(
  2523. comp_ctx, func_ctx, *frame_ip++,
  2524. SIMD_i16x8_extract_lane_s == opcode))
  2525. return false;
  2526. break;
  2527. }
  2528. case SIMD_i16x8_replace_lane:
  2529. {
  2530. if (!aot_compile_simd_replace_i16x8(comp_ctx, func_ctx,
  2531. *frame_ip++))
  2532. return false;
  2533. break;
  2534. }
  2535. case SIMD_i32x4_extract_lane:
  2536. {
  2537. if (!aot_compile_simd_extract_i32x4(comp_ctx, func_ctx,
  2538. *frame_ip++))
  2539. return false;
  2540. break;
  2541. }
  2542. case SIMD_i32x4_replace_lane:
  2543. {
  2544. if (!aot_compile_simd_replace_i32x4(comp_ctx, func_ctx,
  2545. *frame_ip++))
  2546. return false;
  2547. break;
  2548. }
  2549. case SIMD_i64x2_extract_lane:
  2550. {
  2551. if (!aot_compile_simd_extract_i64x2(comp_ctx, func_ctx,
  2552. *frame_ip++))
  2553. return false;
  2554. break;
  2555. }
  2556. case SIMD_i64x2_replace_lane:
  2557. {
  2558. if (!aot_compile_simd_replace_i64x2(comp_ctx, func_ctx,
  2559. *frame_ip++))
  2560. return false;
  2561. break;
  2562. }
  2563. case SIMD_f32x4_extract_lane:
  2564. {
  2565. if (!aot_compile_simd_extract_f32x4(comp_ctx, func_ctx,
  2566. *frame_ip++))
  2567. return false;
  2568. break;
  2569. }
  2570. case SIMD_f32x4_replace_lane:
  2571. {
  2572. if (!aot_compile_simd_replace_f32x4(comp_ctx, func_ctx,
  2573. *frame_ip++))
  2574. return false;
  2575. break;
  2576. }
  2577. case SIMD_f64x2_extract_lane:
  2578. {
  2579. if (!aot_compile_simd_extract_f64x2(comp_ctx, func_ctx,
  2580. *frame_ip++))
  2581. return false;
  2582. break;
  2583. }
  2584. case SIMD_f64x2_replace_lane:
  2585. {
  2586. if (!aot_compile_simd_replace_f64x2(comp_ctx, func_ctx,
  2587. *frame_ip++))
  2588. return false;
  2589. break;
  2590. }
  2591. /* i8x16 Cmp */
  2592. case SIMD_i8x16_eq:
  2593. case SIMD_i8x16_ne:
  2594. case SIMD_i8x16_lt_s:
  2595. case SIMD_i8x16_lt_u:
  2596. case SIMD_i8x16_gt_s:
  2597. case SIMD_i8x16_gt_u:
  2598. case SIMD_i8x16_le_s:
  2599. case SIMD_i8x16_le_u:
  2600. case SIMD_i8x16_ge_s:
  2601. case SIMD_i8x16_ge_u:
  2602. {
  2603. if (!aot_compile_simd_i8x16_compare(
  2604. comp_ctx, func_ctx,
  2605. INT_EQ + opcode - SIMD_i8x16_eq))
  2606. return false;
  2607. break;
  2608. }
  2609. /* i16x8 Cmp */
  2610. case SIMD_i16x8_eq:
  2611. case SIMD_i16x8_ne:
  2612. case SIMD_i16x8_lt_s:
  2613. case SIMD_i16x8_lt_u:
  2614. case SIMD_i16x8_gt_s:
  2615. case SIMD_i16x8_gt_u:
  2616. case SIMD_i16x8_le_s:
  2617. case SIMD_i16x8_le_u:
  2618. case SIMD_i16x8_ge_s:
  2619. case SIMD_i16x8_ge_u:
  2620. {
  2621. if (!aot_compile_simd_i16x8_compare(
  2622. comp_ctx, func_ctx,
  2623. INT_EQ + opcode - SIMD_i16x8_eq))
  2624. return false;
  2625. break;
  2626. }
  2627. /* i32x4 Cmp */
  2628. case SIMD_i32x4_eq:
  2629. case SIMD_i32x4_ne:
  2630. case SIMD_i32x4_lt_s:
  2631. case SIMD_i32x4_lt_u:
  2632. case SIMD_i32x4_gt_s:
  2633. case SIMD_i32x4_gt_u:
  2634. case SIMD_i32x4_le_s:
  2635. case SIMD_i32x4_le_u:
  2636. case SIMD_i32x4_ge_s:
  2637. case SIMD_i32x4_ge_u:
  2638. {
  2639. if (!aot_compile_simd_i32x4_compare(
  2640. comp_ctx, func_ctx,
  2641. INT_EQ + opcode - SIMD_i32x4_eq))
  2642. return false;
  2643. break;
  2644. }
  2645. /* f32x4 Cmp */
  2646. case SIMD_f32x4_eq:
  2647. case SIMD_f32x4_ne:
  2648. case SIMD_f32x4_lt:
  2649. case SIMD_f32x4_gt:
  2650. case SIMD_f32x4_le:
  2651. case SIMD_f32x4_ge:
  2652. {
  2653. if (!aot_compile_simd_f32x4_compare(
  2654. comp_ctx, func_ctx,
  2655. FLOAT_EQ + opcode - SIMD_f32x4_eq))
  2656. return false;
  2657. break;
  2658. }
  2659. /* f64x2 Cmp */
  2660. case SIMD_f64x2_eq:
  2661. case SIMD_f64x2_ne:
  2662. case SIMD_f64x2_lt:
  2663. case SIMD_f64x2_gt:
  2664. case SIMD_f64x2_le:
  2665. case SIMD_f64x2_ge:
  2666. {
  2667. if (!aot_compile_simd_f64x2_compare(
  2668. comp_ctx, func_ctx,
  2669. FLOAT_EQ + opcode - SIMD_f64x2_eq))
  2670. return false;
  2671. break;
  2672. }
  2673. /* v128 Op */
  2674. case SIMD_v128_not:
  2675. case SIMD_v128_and:
  2676. case SIMD_v128_andnot:
  2677. case SIMD_v128_or:
  2678. case SIMD_v128_xor:
  2679. case SIMD_v128_bitselect:
  2680. {
  2681. if (!aot_compile_simd_v128_bitwise(comp_ctx, func_ctx,
  2682. V128_NOT + opcode
  2683. - SIMD_v128_not))
  2684. return false;
  2685. break;
  2686. }
  2687. case SIMD_v128_any_true:
  2688. {
  2689. if (!aot_compile_simd_v128_any_true(comp_ctx, func_ctx))
  2690. return false;
  2691. break;
  2692. }
  2693. /* Load Lane Op */
  2694. case SIMD_v128_load8_lane:
  2695. case SIMD_v128_load16_lane:
  2696. case SIMD_v128_load32_lane:
  2697. case SIMD_v128_load64_lane:
  2698. {
  2699. read_leb_uint32(frame_ip, frame_ip_end, align);
  2700. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2701. if (!aot_compile_simd_load_lane(comp_ctx, func_ctx,
  2702. opcode, align, offset,
  2703. *frame_ip++))
  2704. return false;
  2705. break;
  2706. }
  2707. case SIMD_v128_store8_lane:
  2708. case SIMD_v128_store16_lane:
  2709. case SIMD_v128_store32_lane:
  2710. case SIMD_v128_store64_lane:
  2711. {
  2712. read_leb_uint32(frame_ip, frame_ip_end, align);
  2713. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2714. if (!aot_compile_simd_store_lane(comp_ctx, func_ctx,
  2715. opcode, align, offset,
  2716. *frame_ip++))
  2717. return false;
  2718. break;
  2719. }
  2720. case SIMD_v128_load32_zero:
  2721. case SIMD_v128_load64_zero:
  2722. {
  2723. read_leb_uint32(frame_ip, frame_ip_end, align);
  2724. read_leb_uint32(frame_ip, frame_ip_end, offset);
  2725. if (!aot_compile_simd_load_zero(comp_ctx, func_ctx,
  2726. opcode, align, offset))
  2727. return false;
  2728. break;
  2729. }
  2730. /* Float conversion */
  2731. case SIMD_f32x4_demote_f64x2_zero:
  2732. {
  2733. if (!aot_compile_simd_f64x2_demote(comp_ctx, func_ctx))
  2734. return false;
  2735. break;
  2736. }
  2737. case SIMD_f64x2_promote_low_f32x4_zero:
  2738. {
  2739. if (!aot_compile_simd_f32x4_promote(comp_ctx, func_ctx))
  2740. return false;
  2741. break;
  2742. }
  2743. /* i8x16 Op */
  2744. case SIMD_i8x16_abs:
  2745. {
  2746. if (!aot_compile_simd_i8x16_abs(comp_ctx, func_ctx))
  2747. return false;
  2748. break;
  2749. }
  2750. case SIMD_i8x16_neg:
  2751. {
  2752. if (!aot_compile_simd_i8x16_neg(comp_ctx, func_ctx))
  2753. return false;
  2754. break;
  2755. }
  2756. case SIMD_i8x16_popcnt:
  2757. {
  2758. if (!aot_compile_simd_i8x16_popcnt(comp_ctx, func_ctx))
  2759. return false;
  2760. break;
  2761. }
  2762. case SIMD_i8x16_all_true:
  2763. {
  2764. if (!aot_compile_simd_i8x16_all_true(comp_ctx,
  2765. func_ctx))
  2766. return false;
  2767. break;
  2768. }
  2769. case SIMD_i8x16_bitmask:
  2770. {
  2771. if (!aot_compile_simd_i8x16_bitmask(comp_ctx, func_ctx))
  2772. return false;
  2773. break;
  2774. }
  2775. case SIMD_i8x16_narrow_i16x8_s:
  2776. case SIMD_i8x16_narrow_i16x8_u:
  2777. {
  2778. if (!aot_compile_simd_i8x16_narrow_i16x8(
  2779. comp_ctx, func_ctx,
  2780. (opcode == SIMD_i8x16_narrow_i16x8_s)))
  2781. return false;
  2782. break;
  2783. }
  2784. case SIMD_f32x4_ceil:
  2785. {
  2786. if (!aot_compile_simd_f32x4_ceil(comp_ctx, func_ctx))
  2787. return false;
  2788. break;
  2789. }
  2790. case SIMD_f32x4_floor:
  2791. {
  2792. if (!aot_compile_simd_f32x4_floor(comp_ctx, func_ctx))
  2793. return false;
  2794. break;
  2795. }
  2796. case SIMD_f32x4_trunc:
  2797. {
  2798. if (!aot_compile_simd_f32x4_trunc(comp_ctx, func_ctx))
  2799. return false;
  2800. break;
  2801. }
  2802. case SIMD_f32x4_nearest:
  2803. {
  2804. if (!aot_compile_simd_f32x4_nearest(comp_ctx, func_ctx))
  2805. return false;
  2806. break;
  2807. }
  2808. case SIMD_i8x16_shl:
  2809. case SIMD_i8x16_shr_s:
  2810. case SIMD_i8x16_shr_u:
  2811. {
  2812. if (!aot_compile_simd_i8x16_shift(comp_ctx, func_ctx,
  2813. INT_SHL + opcode
  2814. - SIMD_i8x16_shl))
  2815. return false;
  2816. break;
  2817. }
  2818. case SIMD_i8x16_add:
  2819. {
  2820. if (!aot_compile_simd_i8x16_arith(comp_ctx, func_ctx,
  2821. V128_ADD))
  2822. return false;
  2823. break;
  2824. }
  2825. case SIMD_i8x16_add_sat_s:
  2826. case SIMD_i8x16_add_sat_u:
  2827. {
  2828. if (!aot_compile_simd_i8x16_saturate(
  2829. comp_ctx, func_ctx, V128_ADD,
  2830. opcode == SIMD_i8x16_add_sat_s))
  2831. return false;
  2832. break;
  2833. }
  2834. case SIMD_i8x16_sub:
  2835. {
  2836. if (!aot_compile_simd_i8x16_arith(comp_ctx, func_ctx,
  2837. V128_SUB))
  2838. return false;
  2839. break;
  2840. }
  2841. case SIMD_i8x16_sub_sat_s:
  2842. case SIMD_i8x16_sub_sat_u:
  2843. {
  2844. if (!aot_compile_simd_i8x16_saturate(
  2845. comp_ctx, func_ctx, V128_SUB,
  2846. opcode == SIMD_i8x16_sub_sat_s))
  2847. return false;
  2848. break;
  2849. }
  2850. case SIMD_f64x2_ceil:
  2851. {
  2852. if (!aot_compile_simd_f64x2_ceil(comp_ctx, func_ctx))
  2853. return false;
  2854. break;
  2855. }
  2856. case SIMD_f64x2_floor:
  2857. {
  2858. if (!aot_compile_simd_f64x2_floor(comp_ctx, func_ctx))
  2859. return false;
  2860. break;
  2861. }
  2862. case SIMD_i8x16_min_s:
  2863. case SIMD_i8x16_min_u:
  2864. {
  2865. if (!aot_compile_simd_i8x16_cmp(
  2866. comp_ctx, func_ctx, V128_MIN,
  2867. opcode == SIMD_i8x16_min_s))
  2868. return false;
  2869. break;
  2870. }
  2871. case SIMD_i8x16_max_s:
  2872. case SIMD_i8x16_max_u:
  2873. {
  2874. if (!aot_compile_simd_i8x16_cmp(
  2875. comp_ctx, func_ctx, V128_MAX,
  2876. opcode == SIMD_i8x16_max_s))
  2877. return false;
  2878. break;
  2879. }
  2880. case SIMD_f64x2_trunc:
  2881. {
  2882. if (!aot_compile_simd_f64x2_trunc(comp_ctx, func_ctx))
  2883. return false;
  2884. break;
  2885. }
  2886. case SIMD_i8x16_avgr_u:
  2887. {
  2888. if (!aot_compile_simd_i8x16_avgr_u(comp_ctx, func_ctx))
  2889. return false;
  2890. break;
  2891. }
  2892. case SIMD_i16x8_extadd_pairwise_i8x16_s:
  2893. case SIMD_i16x8_extadd_pairwise_i8x16_u:
  2894. {
  2895. if (!aot_compile_simd_i16x8_extadd_pairwise_i8x16(
  2896. comp_ctx, func_ctx,
  2897. SIMD_i16x8_extadd_pairwise_i8x16_s == opcode))
  2898. return false;
  2899. break;
  2900. }
  2901. case SIMD_i32x4_extadd_pairwise_i16x8_s:
  2902. case SIMD_i32x4_extadd_pairwise_i16x8_u:
  2903. {
  2904. if (!aot_compile_simd_i32x4_extadd_pairwise_i16x8(
  2905. comp_ctx, func_ctx,
  2906. SIMD_i32x4_extadd_pairwise_i16x8_s == opcode))
  2907. return false;
  2908. break;
  2909. }
  2910. /* i16x8 Op */
  2911. case SIMD_i16x8_abs:
  2912. {
  2913. if (!aot_compile_simd_i16x8_abs(comp_ctx, func_ctx))
  2914. return false;
  2915. break;
  2916. }
  2917. case SIMD_i16x8_neg:
  2918. {
  2919. if (!aot_compile_simd_i16x8_neg(comp_ctx, func_ctx))
  2920. return false;
  2921. break;
  2922. }
  2923. case SIMD_i16x8_q15mulr_sat_s:
  2924. {
  2925. if (!aot_compile_simd_i16x8_q15mulr_sat(comp_ctx,
  2926. func_ctx))
  2927. return false;
  2928. break;
  2929. }
  2930. case SIMD_i16x8_all_true:
  2931. {
  2932. if (!aot_compile_simd_i16x8_all_true(comp_ctx,
  2933. func_ctx))
  2934. return false;
  2935. break;
  2936. }
  2937. case SIMD_i16x8_bitmask:
  2938. {
  2939. if (!aot_compile_simd_i16x8_bitmask(comp_ctx, func_ctx))
  2940. return false;
  2941. break;
  2942. }
  2943. case SIMD_i16x8_narrow_i32x4_s:
  2944. case SIMD_i16x8_narrow_i32x4_u:
  2945. {
  2946. if (!aot_compile_simd_i16x8_narrow_i32x4(
  2947. comp_ctx, func_ctx,
  2948. SIMD_i16x8_narrow_i32x4_s == opcode))
  2949. return false;
  2950. break;
  2951. }
  2952. case SIMD_i16x8_extend_low_i8x16_s:
  2953. case SIMD_i16x8_extend_high_i8x16_s:
  2954. {
  2955. if (!aot_compile_simd_i16x8_extend_i8x16(
  2956. comp_ctx, func_ctx,
  2957. SIMD_i16x8_extend_low_i8x16_s == opcode, true))
  2958. return false;
  2959. break;
  2960. }
  2961. case SIMD_i16x8_extend_low_i8x16_u:
  2962. case SIMD_i16x8_extend_high_i8x16_u:
  2963. {
  2964. if (!aot_compile_simd_i16x8_extend_i8x16(
  2965. comp_ctx, func_ctx,
  2966. SIMD_i16x8_extend_low_i8x16_u == opcode, false))
  2967. return false;
  2968. break;
  2969. }
  2970. case SIMD_i16x8_shl:
  2971. case SIMD_i16x8_shr_s:
  2972. case SIMD_i16x8_shr_u:
  2973. {
  2974. if (!aot_compile_simd_i16x8_shift(comp_ctx, func_ctx,
  2975. INT_SHL + opcode
  2976. - SIMD_i16x8_shl))
  2977. return false;
  2978. break;
  2979. }
  2980. case SIMD_i16x8_add:
  2981. {
  2982. if (!aot_compile_simd_i16x8_arith(comp_ctx, func_ctx,
  2983. V128_ADD))
  2984. return false;
  2985. break;
  2986. }
  2987. case SIMD_i16x8_add_sat_s:
  2988. case SIMD_i16x8_add_sat_u:
  2989. {
  2990. if (!aot_compile_simd_i16x8_saturate(
  2991. comp_ctx, func_ctx, V128_ADD,
  2992. opcode == SIMD_i16x8_add_sat_s ? true : false))
  2993. return false;
  2994. break;
  2995. }
  2996. case SIMD_i16x8_sub:
  2997. {
  2998. if (!aot_compile_simd_i16x8_arith(comp_ctx, func_ctx,
  2999. V128_SUB))
  3000. return false;
  3001. break;
  3002. }
  3003. case SIMD_i16x8_sub_sat_s:
  3004. case SIMD_i16x8_sub_sat_u:
  3005. {
  3006. if (!aot_compile_simd_i16x8_saturate(
  3007. comp_ctx, func_ctx, V128_SUB,
  3008. opcode == SIMD_i16x8_sub_sat_s ? true : false))
  3009. return false;
  3010. break;
  3011. }
  3012. case SIMD_f64x2_nearest:
  3013. {
  3014. if (!aot_compile_simd_f64x2_nearest(comp_ctx, func_ctx))
  3015. return false;
  3016. break;
  3017. }
  3018. case SIMD_i16x8_mul:
  3019. {
  3020. if (!aot_compile_simd_i16x8_arith(comp_ctx, func_ctx,
  3021. V128_MUL))
  3022. return false;
  3023. break;
  3024. }
  3025. case SIMD_i16x8_min_s:
  3026. case SIMD_i16x8_min_u:
  3027. {
  3028. if (!aot_compile_simd_i16x8_cmp(
  3029. comp_ctx, func_ctx, V128_MIN,
  3030. opcode == SIMD_i16x8_min_s))
  3031. return false;
  3032. break;
  3033. }
  3034. case SIMD_i16x8_max_s:
  3035. case SIMD_i16x8_max_u:
  3036. {
  3037. if (!aot_compile_simd_i16x8_cmp(
  3038. comp_ctx, func_ctx, V128_MAX,
  3039. opcode == SIMD_i16x8_max_s))
  3040. return false;
  3041. break;
  3042. }
  3043. case SIMD_i16x8_avgr_u:
  3044. {
  3045. if (!aot_compile_simd_i16x8_avgr_u(comp_ctx, func_ctx))
  3046. return false;
  3047. break;
  3048. }
  3049. case SIMD_i16x8_extmul_low_i8x16_s:
  3050. case SIMD_i16x8_extmul_high_i8x16_s:
  3051. {
  3052. if (!(aot_compile_simd_i16x8_extmul_i8x16(
  3053. comp_ctx, func_ctx,
  3054. SIMD_i16x8_extmul_low_i8x16_s == opcode, true)))
  3055. return false;
  3056. break;
  3057. }
  3058. case SIMD_i16x8_extmul_low_i8x16_u:
  3059. case SIMD_i16x8_extmul_high_i8x16_u:
  3060. {
  3061. if (!(aot_compile_simd_i16x8_extmul_i8x16(
  3062. comp_ctx, func_ctx,
  3063. SIMD_i16x8_extmul_low_i8x16_u == opcode,
  3064. false)))
  3065. return false;
  3066. break;
  3067. }
  3068. /* i32x4 Op */
  3069. case SIMD_i32x4_abs:
  3070. {
  3071. if (!aot_compile_simd_i32x4_abs(comp_ctx, func_ctx))
  3072. return false;
  3073. break;
  3074. }
  3075. case SIMD_i32x4_neg:
  3076. {
  3077. if (!aot_compile_simd_i32x4_neg(comp_ctx, func_ctx))
  3078. return false;
  3079. break;
  3080. }
  3081. case SIMD_i32x4_all_true:
  3082. {
  3083. if (!aot_compile_simd_i32x4_all_true(comp_ctx,
  3084. func_ctx))
  3085. return false;
  3086. break;
  3087. }
  3088. case SIMD_i32x4_bitmask:
  3089. {
  3090. if (!aot_compile_simd_i32x4_bitmask(comp_ctx, func_ctx))
  3091. return false;
  3092. break;
  3093. }
  3094. case SIMD_i32x4_extend_low_i16x8_s:
  3095. case SIMD_i32x4_extend_high_i16x8_s:
  3096. {
  3097. if (!aot_compile_simd_i32x4_extend_i16x8(
  3098. comp_ctx, func_ctx,
  3099. SIMD_i32x4_extend_low_i16x8_s == opcode, true))
  3100. return false;
  3101. break;
  3102. }
  3103. case SIMD_i32x4_extend_low_i16x8_u:
  3104. case SIMD_i32x4_extend_high_i16x8_u:
  3105. {
  3106. if (!aot_compile_simd_i32x4_extend_i16x8(
  3107. comp_ctx, func_ctx,
  3108. SIMD_i32x4_extend_low_i16x8_u == opcode, false))
  3109. return false;
  3110. break;
  3111. }
  3112. case SIMD_i32x4_shl:
  3113. case SIMD_i32x4_shr_s:
  3114. case SIMD_i32x4_shr_u:
  3115. {
  3116. if (!aot_compile_simd_i32x4_shift(comp_ctx, func_ctx,
  3117. INT_SHL + opcode
  3118. - SIMD_i32x4_shl))
  3119. return false;
  3120. break;
  3121. }
  3122. case SIMD_i32x4_add:
  3123. {
  3124. if (!aot_compile_simd_i32x4_arith(comp_ctx, func_ctx,
  3125. V128_ADD))
  3126. return false;
  3127. break;
  3128. }
  3129. case SIMD_i32x4_sub:
  3130. {
  3131. if (!aot_compile_simd_i32x4_arith(comp_ctx, func_ctx,
  3132. V128_SUB))
  3133. return false;
  3134. break;
  3135. }
  3136. case SIMD_i32x4_mul:
  3137. {
  3138. if (!aot_compile_simd_i32x4_arith(comp_ctx, func_ctx,
  3139. V128_MUL))
  3140. return false;
  3141. break;
  3142. }
  3143. case SIMD_i32x4_min_s:
  3144. case SIMD_i32x4_min_u:
  3145. {
  3146. if (!aot_compile_simd_i32x4_cmp(
  3147. comp_ctx, func_ctx, V128_MIN,
  3148. SIMD_i32x4_min_s == opcode))
  3149. return false;
  3150. break;
  3151. }
  3152. case SIMD_i32x4_max_s:
  3153. case SIMD_i32x4_max_u:
  3154. {
  3155. if (!aot_compile_simd_i32x4_cmp(
  3156. comp_ctx, func_ctx, V128_MAX,
  3157. SIMD_i32x4_max_s == opcode))
  3158. return false;
  3159. break;
  3160. }
  3161. case SIMD_i32x4_dot_i16x8_s:
  3162. {
  3163. if (!aot_compile_simd_i32x4_dot_i16x8(comp_ctx,
  3164. func_ctx))
  3165. return false;
  3166. break;
  3167. }
  3168. case SIMD_i32x4_extmul_low_i16x8_s:
  3169. case SIMD_i32x4_extmul_high_i16x8_s:
  3170. {
  3171. if (!aot_compile_simd_i32x4_extmul_i16x8(
  3172. comp_ctx, func_ctx,
  3173. SIMD_i32x4_extmul_low_i16x8_s == opcode, true))
  3174. return false;
  3175. break;
  3176. }
  3177. case SIMD_i32x4_extmul_low_i16x8_u:
  3178. case SIMD_i32x4_extmul_high_i16x8_u:
  3179. {
  3180. if (!aot_compile_simd_i32x4_extmul_i16x8(
  3181. comp_ctx, func_ctx,
  3182. SIMD_i32x4_extmul_low_i16x8_u == opcode, false))
  3183. return false;
  3184. break;
  3185. }
  3186. /* i64x2 Op */
  3187. case SIMD_i64x2_abs:
  3188. {
  3189. if (!aot_compile_simd_i64x2_abs(comp_ctx, func_ctx))
  3190. return false;
  3191. break;
  3192. }
  3193. case SIMD_i64x2_neg:
  3194. {
  3195. if (!aot_compile_simd_i64x2_neg(comp_ctx, func_ctx))
  3196. return false;
  3197. break;
  3198. }
  3199. case SIMD_i64x2_all_true:
  3200. {
  3201. if (!aot_compile_simd_i64x2_all_true(comp_ctx,
  3202. func_ctx))
  3203. return false;
  3204. break;
  3205. }
  3206. case SIMD_i64x2_bitmask:
  3207. {
  3208. if (!aot_compile_simd_i64x2_bitmask(comp_ctx, func_ctx))
  3209. return false;
  3210. break;
  3211. }
  3212. case SIMD_i64x2_extend_low_i32x4_s:
  3213. case SIMD_i64x2_extend_high_i32x4_s:
  3214. {
  3215. if (!aot_compile_simd_i64x2_extend_i32x4(
  3216. comp_ctx, func_ctx,
  3217. SIMD_i64x2_extend_low_i32x4_s == opcode, true))
  3218. return false;
  3219. break;
  3220. }
  3221. case SIMD_i64x2_extend_low_i32x4_u:
  3222. case SIMD_i64x2_extend_high_i32x4_u:
  3223. {
  3224. if (!aot_compile_simd_i64x2_extend_i32x4(
  3225. comp_ctx, func_ctx,
  3226. SIMD_i64x2_extend_low_i32x4_u == opcode, false))
  3227. return false;
  3228. break;
  3229. }
  3230. case SIMD_i64x2_shl:
  3231. case SIMD_i64x2_shr_s:
  3232. case SIMD_i64x2_shr_u:
  3233. {
  3234. if (!aot_compile_simd_i64x2_shift(comp_ctx, func_ctx,
  3235. INT_SHL + opcode
  3236. - SIMD_i64x2_shl))
  3237. return false;
  3238. break;
  3239. }
  3240. case SIMD_i64x2_add:
  3241. {
  3242. if (!aot_compile_simd_i64x2_arith(comp_ctx, func_ctx,
  3243. V128_ADD))
  3244. return false;
  3245. break;
  3246. }
  3247. case SIMD_i64x2_sub:
  3248. {
  3249. if (!aot_compile_simd_i64x2_arith(comp_ctx, func_ctx,
  3250. V128_SUB))
  3251. return false;
  3252. break;
  3253. }
  3254. case SIMD_i64x2_mul:
  3255. {
  3256. if (!aot_compile_simd_i64x2_arith(comp_ctx, func_ctx,
  3257. V128_MUL))
  3258. return false;
  3259. break;
  3260. }
  3261. case SIMD_i64x2_eq:
  3262. case SIMD_i64x2_ne:
  3263. case SIMD_i64x2_lt_s:
  3264. case SIMD_i64x2_gt_s:
  3265. case SIMD_i64x2_le_s:
  3266. case SIMD_i64x2_ge_s:
  3267. {
  3268. IntCond icond[] = { INT_EQ, INT_NE, INT_LT_S,
  3269. INT_GT_S, INT_LE_S, INT_GE_S };
  3270. if (!aot_compile_simd_i64x2_compare(
  3271. comp_ctx, func_ctx,
  3272. icond[opcode - SIMD_i64x2_eq]))
  3273. return false;
  3274. break;
  3275. }
  3276. case SIMD_i64x2_extmul_low_i32x4_s:
  3277. case SIMD_i64x2_extmul_high_i32x4_s:
  3278. {
  3279. if (!aot_compile_simd_i64x2_extmul_i32x4(
  3280. comp_ctx, func_ctx,
  3281. SIMD_i64x2_extmul_low_i32x4_s == opcode, true))
  3282. return false;
  3283. break;
  3284. }
  3285. case SIMD_i64x2_extmul_low_i32x4_u:
  3286. case SIMD_i64x2_extmul_high_i32x4_u:
  3287. {
  3288. if (!aot_compile_simd_i64x2_extmul_i32x4(
  3289. comp_ctx, func_ctx,
  3290. SIMD_i64x2_extmul_low_i32x4_u == opcode, false))
  3291. return false;
  3292. break;
  3293. }
  3294. /* f32x4 Op */
  3295. case SIMD_f32x4_abs:
  3296. {
  3297. if (!aot_compile_simd_f32x4_abs(comp_ctx, func_ctx))
  3298. return false;
  3299. break;
  3300. }
  3301. case SIMD_f32x4_neg:
  3302. {
  3303. if (!aot_compile_simd_f32x4_neg(comp_ctx, func_ctx))
  3304. return false;
  3305. break;
  3306. }
  3307. case SIMD_f32x4_sqrt:
  3308. {
  3309. if (!aot_compile_simd_f32x4_sqrt(comp_ctx, func_ctx))
  3310. return false;
  3311. break;
  3312. }
  3313. case SIMD_f32x4_add:
  3314. case SIMD_f32x4_sub:
  3315. case SIMD_f32x4_mul:
  3316. case SIMD_f32x4_div:
  3317. {
  3318. if (!aot_compile_simd_f32x4_arith(comp_ctx, func_ctx,
  3319. FLOAT_ADD + opcode
  3320. - SIMD_f32x4_add))
  3321. return false;
  3322. break;
  3323. }
  3324. case SIMD_f32x4_min:
  3325. case SIMD_f32x4_max:
  3326. {
  3327. if (!aot_compile_simd_f32x4_min_max(
  3328. comp_ctx, func_ctx, SIMD_f32x4_min == opcode))
  3329. return false;
  3330. break;
  3331. }
  3332. case SIMD_f32x4_pmin:
  3333. case SIMD_f32x4_pmax:
  3334. {
  3335. if (!aot_compile_simd_f32x4_pmin_pmax(
  3336. comp_ctx, func_ctx, SIMD_f32x4_pmin == opcode))
  3337. return false;
  3338. break;
  3339. }
  3340. /* f64x2 Op */
  3341. case SIMD_f64x2_abs:
  3342. {
  3343. if (!aot_compile_simd_f64x2_abs(comp_ctx, func_ctx))
  3344. return false;
  3345. break;
  3346. }
  3347. case SIMD_f64x2_neg:
  3348. {
  3349. if (!aot_compile_simd_f64x2_neg(comp_ctx, func_ctx))
  3350. return false;
  3351. break;
  3352. }
  3353. case SIMD_f64x2_sqrt:
  3354. {
  3355. if (!aot_compile_simd_f64x2_sqrt(comp_ctx, func_ctx))
  3356. return false;
  3357. break;
  3358. }
  3359. case SIMD_f64x2_add:
  3360. case SIMD_f64x2_sub:
  3361. case SIMD_f64x2_mul:
  3362. case SIMD_f64x2_div:
  3363. {
  3364. if (!aot_compile_simd_f64x2_arith(comp_ctx, func_ctx,
  3365. FLOAT_ADD + opcode
  3366. - SIMD_f64x2_add))
  3367. return false;
  3368. break;
  3369. }
  3370. case SIMD_f64x2_min:
  3371. case SIMD_f64x2_max:
  3372. {
  3373. if (!aot_compile_simd_f64x2_min_max(
  3374. comp_ctx, func_ctx, SIMD_f64x2_min == opcode))
  3375. return false;
  3376. break;
  3377. }
  3378. case SIMD_f64x2_pmin:
  3379. case SIMD_f64x2_pmax:
  3380. {
  3381. if (!aot_compile_simd_f64x2_pmin_pmax(
  3382. comp_ctx, func_ctx, SIMD_f64x2_pmin == opcode))
  3383. return false;
  3384. break;
  3385. }
  3386. /* Conversion Op */
  3387. case SIMD_i32x4_trunc_sat_f32x4_s:
  3388. case SIMD_i32x4_trunc_sat_f32x4_u:
  3389. {
  3390. if (!aot_compile_simd_i32x4_trunc_sat_f32x4(
  3391. comp_ctx, func_ctx,
  3392. SIMD_i32x4_trunc_sat_f32x4_s == opcode))
  3393. return false;
  3394. break;
  3395. }
  3396. case SIMD_f32x4_convert_i32x4_s:
  3397. case SIMD_f32x4_convert_i32x4_u:
  3398. {
  3399. if (!aot_compile_simd_f32x4_convert_i32x4(
  3400. comp_ctx, func_ctx,
  3401. SIMD_f32x4_convert_i32x4_s == opcode))
  3402. return false;
  3403. break;
  3404. }
  3405. case SIMD_i32x4_trunc_sat_f64x2_s_zero:
  3406. case SIMD_i32x4_trunc_sat_f64x2_u_zero:
  3407. {
  3408. if (!aot_compile_simd_i32x4_trunc_sat_f64x2(
  3409. comp_ctx, func_ctx,
  3410. SIMD_i32x4_trunc_sat_f64x2_s_zero == opcode))
  3411. return false;
  3412. break;
  3413. }
  3414. case SIMD_f64x2_convert_low_i32x4_s:
  3415. case SIMD_f64x2_convert_low_i32x4_u:
  3416. {
  3417. if (!aot_compile_simd_f64x2_convert_i32x4(
  3418. comp_ctx, func_ctx,
  3419. SIMD_f64x2_convert_low_i32x4_s == opcode))
  3420. return false;
  3421. break;
  3422. }
  3423. default:
  3424. aot_set_last_error("unsupported SIMD opcode");
  3425. return false;
  3426. }
  3427. break;
  3428. }
  3429. #endif /* end of WASM_ENABLE_SIMD */
  3430. default:
  3431. aot_set_last_error("unsupported opcode");
  3432. return false;
  3433. }
  3434. }
  3435. /* Move func_return block to the bottom */
  3436. if (func_ctx->func_return_block) {
  3437. LLVMBasicBlockRef last_block = LLVMGetLastBasicBlock(func_ctx->func);
  3438. if (last_block != func_ctx->func_return_block)
  3439. LLVMMoveBasicBlockAfter(func_ctx->func_return_block, last_block);
  3440. }
  3441. /* Move got_exception block to the bottom */
  3442. if (func_ctx->got_exception_block) {
  3443. LLVMBasicBlockRef last_block = LLVMGetLastBasicBlock(func_ctx->func);
  3444. if (last_block != func_ctx->got_exception_block)
  3445. LLVMMoveBasicBlockAfter(func_ctx->got_exception_block, last_block);
  3446. }
  3447. return true;
  3448. #if WASM_ENABLE_SIMD != 0
  3449. unsupport_simd:
  3450. aot_set_last_error("SIMD instruction was found, "
  3451. "try removing --disable-simd option");
  3452. return false;
  3453. #endif
  3454. #if WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0
  3455. unsupport_ref_types:
  3456. aot_set_last_error("reference type instruction was found, "
  3457. "try removing --disable-ref-types option "
  3458. "or adding --enable-gc option");
  3459. return false;
  3460. #endif
  3461. #if WASM_ENABLE_GC != 0
  3462. unsupport_gc:
  3463. aot_set_last_error("GC instruction was found, "
  3464. "try adding --enable-gc option");
  3465. return false;
  3466. #endif
  3467. #if WASM_ENABLE_REF_TYPES != 0 || WASM_ENABLE_GC != 0
  3468. unsupport_gc_and_ref_types:
  3469. aot_set_last_error(
  3470. "reference type or gc instruction was found, try removing "
  3471. "--disable-ref-types option or adding --enable-gc option");
  3472. return false;
  3473. #endif
  3474. #if WASM_ENABLE_BULK_MEMORY != 0
  3475. unsupport_bulk_memory:
  3476. aot_set_last_error("bulk memory instruction was found, "
  3477. "try removing --disable-bulk-memory option");
  3478. return false;
  3479. #endif
  3480. fail:
  3481. return false;
  3482. }
  3483. static bool
  3484. verify_module(AOTCompContext *comp_ctx)
  3485. {
  3486. char *msg = NULL;
  3487. bool ret;
  3488. ret = LLVMVerifyModule(comp_ctx->module, LLVMPrintMessageAction, &msg);
  3489. if (!ret && msg) {
  3490. if (msg[0] != '\0') {
  3491. aot_set_last_error(msg);
  3492. LLVMDisposeMessage(msg);
  3493. return false;
  3494. }
  3495. LLVMDisposeMessage(msg);
  3496. }
  3497. return true;
  3498. }
  3499. bool
  3500. aot_compile_wasm(AOTCompContext *comp_ctx)
  3501. {
  3502. uint32 i;
  3503. if (!aot_validate_wasm(comp_ctx)) {
  3504. return false;
  3505. }
  3506. bh_print_time("Begin to compile WASM bytecode to LLVM IR");
  3507. for (i = 0; i < comp_ctx->func_ctx_count; i++) {
  3508. if (!aot_compile_func(comp_ctx, i)) {
  3509. return false;
  3510. }
  3511. }
  3512. #if WASM_ENABLE_DEBUG_AOT != 0
  3513. LLVMDIBuilderFinalize(comp_ctx->debug_builder);
  3514. #endif
  3515. /* Disable LLVM module verification for jit mode to speedup
  3516. the compilation process */
  3517. if (!comp_ctx->is_jit_mode) {
  3518. bh_print_time("Begin to verify LLVM module");
  3519. if (!verify_module(comp_ctx)) {
  3520. return false;
  3521. }
  3522. }
  3523. /* Run IR optimization before feeding in ORCJIT and AOT codegen */
  3524. if (comp_ctx->optimize) {
  3525. /* Run passes for AOT/JIT mode.
  3526. TODO: Apply these passes in the do_ir_transform callback of
  3527. TransformLayer when compiling each jit function, so as to
  3528. speedup the launch process. Now there are two issues in the
  3529. JIT: one is memory leak in do_ir_transform, the other is
  3530. possible core dump. */
  3531. bh_print_time("Begin to run llvm optimization passes");
  3532. aot_apply_llvm_new_pass_manager(comp_ctx, comp_ctx->module);
  3533. bh_print_time("Finish llvm optimization passes");
  3534. }
  3535. #ifdef DUMP_MODULE
  3536. LLVMDumpModule(comp_ctx->module);
  3537. os_printf("\n");
  3538. #endif
  3539. if (comp_ctx->is_jit_mode) {
  3540. LLVMErrorRef err;
  3541. LLVMOrcJITDylibRef orc_main_dylib;
  3542. LLVMOrcThreadSafeModuleRef orc_thread_safe_module;
  3543. orc_main_dylib = LLVMOrcLLLazyJITGetMainJITDylib(comp_ctx->orc_jit);
  3544. if (!orc_main_dylib) {
  3545. aot_set_last_error(
  3546. "failed to get orc orc_jit main dynamic library");
  3547. return false;
  3548. }
  3549. orc_thread_safe_module = LLVMOrcCreateNewThreadSafeModule(
  3550. comp_ctx->module, comp_ctx->orc_thread_safe_context);
  3551. if (!orc_thread_safe_module) {
  3552. aot_set_last_error("failed to create thread safe module");
  3553. return false;
  3554. }
  3555. if ((err = LLVMOrcLLLazyJITAddLLVMIRModule(
  3556. comp_ctx->orc_jit, orc_main_dylib, orc_thread_safe_module))) {
  3557. /* If adding the ThreadSafeModule fails then we need to clean it up
  3558. by ourselves, otherwise the orc orc_jit will manage the memory.
  3559. */
  3560. LLVMOrcDisposeThreadSafeModule(orc_thread_safe_module);
  3561. aot_handle_llvm_errmsg("failed to addIRModule", err);
  3562. return false;
  3563. }
  3564. if (comp_ctx->stack_sizes != NULL) {
  3565. LLVMOrcJITTargetAddress addr;
  3566. if ((err = LLVMOrcLLLazyJITLookup(comp_ctx->orc_jit, &addr,
  3567. aot_stack_sizes_alias_name))) {
  3568. aot_handle_llvm_errmsg("failed to look up stack_sizes", err);
  3569. return false;
  3570. }
  3571. comp_ctx->jit_stack_sizes = (uint32 *)addr;
  3572. }
  3573. }
  3574. return true;
  3575. }
  3576. #if !(defined(_WIN32) || defined(_WIN32_))
  3577. char *
  3578. aot_generate_tempfile_name(const char *prefix, const char *extension,
  3579. char *buffer, uint32 len)
  3580. {
  3581. int fd, name_len;
  3582. name_len = snprintf(buffer, len, "%s-XXXXXX", prefix);
  3583. if ((fd = mkstemp(buffer)) <= 0) {
  3584. aot_set_last_error("make temp file failed.");
  3585. return NULL;
  3586. }
  3587. /* close and remove temp file */
  3588. close(fd);
  3589. unlink(buffer);
  3590. /* Check if buffer length is enough */
  3591. /* name_len + '.' + extension + '\0' */
  3592. if (name_len + 1 + strlen(extension) + 1 > len) {
  3593. aot_set_last_error("temp file name too long.");
  3594. return NULL;
  3595. }
  3596. snprintf(buffer + name_len, len - name_len, ".%s", extension);
  3597. return buffer;
  3598. }
  3599. #else
  3600. errno_t
  3601. _mktemp_s(char *nameTemplate, size_t sizeInChars);
  3602. char *
  3603. aot_generate_tempfile_name(const char *prefix, const char *extension,
  3604. char *buffer, uint32 len)
  3605. {
  3606. int name_len;
  3607. name_len = snprintf(buffer, len, "%s-XXXXXX", prefix);
  3608. if (_mktemp_s(buffer, name_len + 1) != 0) {
  3609. return NULL;
  3610. }
  3611. /* Check if buffer length is enough */
  3612. /* name_len + '.' + extension + '\0' */
  3613. if (name_len + 1 + strlen(extension) + 1 > len) {
  3614. aot_set_last_error("temp file name too long.");
  3615. return NULL;
  3616. }
  3617. snprintf(buffer + name_len, len - name_len, ".%s", extension);
  3618. return buffer;
  3619. }
  3620. #endif /* end of !(defined(_WIN32) || defined(_WIN32_)) */
  3621. bool
  3622. aot_emit_llvm_file(AOTCompContext *comp_ctx, const char *file_name)
  3623. {
  3624. char *err = NULL;
  3625. bh_print_time("Begin to emit LLVM IR file");
  3626. if (LLVMPrintModuleToFile(comp_ctx->module, file_name, &err) != 0) {
  3627. if (err) {
  3628. LLVMDisposeMessage(err);
  3629. err = NULL;
  3630. }
  3631. aot_set_last_error("emit llvm ir to file failed.");
  3632. return false;
  3633. }
  3634. return true;
  3635. }
  3636. static bool
  3637. aot_move_file(const char *dest, const char *src)
  3638. {
  3639. FILE *dfp = fopen(dest, "w");
  3640. FILE *sfp = fopen(src, "r");
  3641. size_t rsz;
  3642. char buf[128];
  3643. bool success = false;
  3644. if (dfp == NULL || sfp == NULL) {
  3645. LOG_DEBUG("open error %s %s", dest, src);
  3646. goto fail;
  3647. }
  3648. do {
  3649. rsz = fread(buf, 1, sizeof(buf), sfp);
  3650. if (rsz > 0) {
  3651. size_t wsz = fwrite(buf, 1, rsz, dfp);
  3652. if (wsz < rsz) {
  3653. LOG_DEBUG("write error");
  3654. goto fail;
  3655. }
  3656. }
  3657. if (rsz < sizeof(buf)) {
  3658. if (ferror(sfp)) {
  3659. LOG_DEBUG("read error");
  3660. goto fail;
  3661. }
  3662. }
  3663. } while (rsz > 0);
  3664. success = true;
  3665. fail:
  3666. if (dfp != NULL) {
  3667. if (fclose(dfp)) {
  3668. LOG_DEBUG("close error");
  3669. success = false;
  3670. }
  3671. if (!success) {
  3672. (void)unlink(dest);
  3673. }
  3674. }
  3675. if (sfp != NULL) {
  3676. (void)fclose(sfp);
  3677. }
  3678. if (success) {
  3679. (void)unlink(src);
  3680. }
  3681. return success;
  3682. }
  3683. bool
  3684. aot_emit_object_file(AOTCompContext *comp_ctx, char *file_name)
  3685. {
  3686. char *err = NULL;
  3687. LLVMCodeGenFileType file_type = LLVMObjectFile;
  3688. LLVMTargetRef target = LLVMGetTargetMachineTarget(comp_ctx->target_machine);
  3689. bh_print_time("Begin to emit object file");
  3690. #if !(defined(_WIN32) || defined(_WIN32_))
  3691. if (comp_ctx->external_llc_compiler || comp_ctx->external_asm_compiler) {
  3692. char cmd[1024];
  3693. int ret;
  3694. if (comp_ctx->external_llc_compiler) {
  3695. const char *stack_usage_flag = "";
  3696. char bc_file_name[64];
  3697. char su_file_name[65]; /* See the comment below */
  3698. if (comp_ctx->stack_usage_file != NULL) {
  3699. /*
  3700. * Note: we know the caller uses 64 byte buffer for
  3701. * file_name. It will get 1 byte longer because we
  3702. * replace ".o" with ".su".
  3703. */
  3704. size_t len = strlen(file_name);
  3705. bh_assert(len + 1 <= sizeof(su_file_name));
  3706. bh_assert(len > 3);
  3707. bh_assert(file_name[len - 2] == '.');
  3708. bh_assert(file_name[len - 1] == 'o');
  3709. snprintf(su_file_name, sizeof(su_file_name), "%.*s.su",
  3710. (int)(len - 2), file_name);
  3711. stack_usage_flag = " -fstack-usage";
  3712. }
  3713. if (!aot_generate_tempfile_name("wamrc-bc", "bc", bc_file_name,
  3714. sizeof(bc_file_name))) {
  3715. return false;
  3716. }
  3717. if (LLVMWriteBitcodeToFile(comp_ctx->module, bc_file_name) != 0) {
  3718. aot_set_last_error("emit llvm bitcode file failed.");
  3719. return false;
  3720. }
  3721. snprintf(cmd, sizeof(cmd), "%s%s %s -o %s %s",
  3722. comp_ctx->external_llc_compiler, stack_usage_flag,
  3723. comp_ctx->llc_compiler_flags ? comp_ctx->llc_compiler_flags
  3724. : "-O3 -c",
  3725. file_name, bc_file_name);
  3726. LOG_VERBOSE("invoking external LLC compiler:\n\t%s", cmd);
  3727. ret = system(cmd);
  3728. /* remove temp bitcode file */
  3729. unlink(bc_file_name);
  3730. if (ret != 0) {
  3731. aot_set_last_error("failed to compile LLVM bitcode to obj file "
  3732. "with external LLC compiler.");
  3733. return false;
  3734. }
  3735. if (comp_ctx->stack_usage_file != NULL) {
  3736. /*
  3737. * move the temporary .su file to the specified location.
  3738. *
  3739. * Note: the former is automatically inferred from the output
  3740. * filename (file_name here) by clang.
  3741. *
  3742. * Note: the latter might be user-specified.
  3743. * (wamrc --stack-usage=<file>)
  3744. */
  3745. if (!aot_move_file(comp_ctx->stack_usage_file, su_file_name)) {
  3746. aot_set_last_error("failed to move su file.");
  3747. (void)unlink(su_file_name);
  3748. return false;
  3749. }
  3750. }
  3751. }
  3752. else if (comp_ctx->external_asm_compiler) {
  3753. char asm_file_name[64];
  3754. if (!aot_generate_tempfile_name("wamrc-asm", "s", asm_file_name,
  3755. sizeof(asm_file_name))) {
  3756. return false;
  3757. }
  3758. if (LLVMTargetMachineEmitToFile(comp_ctx->target_machine,
  3759. comp_ctx->module, asm_file_name,
  3760. LLVMAssemblyFile, &err)
  3761. != 0) {
  3762. if (err) {
  3763. LLVMDisposeMessage(err);
  3764. err = NULL;
  3765. }
  3766. aot_set_last_error("emit elf to assembly file failed.");
  3767. return false;
  3768. }
  3769. snprintf(cmd, sizeof(cmd), "%s %s -o %s %s",
  3770. comp_ctx->external_asm_compiler,
  3771. comp_ctx->asm_compiler_flags ? comp_ctx->asm_compiler_flags
  3772. : "-O3 -c",
  3773. file_name, asm_file_name);
  3774. LOG_VERBOSE("invoking external ASM compiler:\n\t%s", cmd);
  3775. ret = system(cmd);
  3776. /* remove temp assembly file */
  3777. unlink(asm_file_name);
  3778. if (ret != 0) {
  3779. aot_set_last_error("failed to compile Assembly file to obj "
  3780. "file with external ASM compiler.");
  3781. return false;
  3782. }
  3783. }
  3784. return true;
  3785. }
  3786. #endif /* end of !(defined(_WIN32) || defined(_WIN32_)) */
  3787. if (!strncmp(LLVMGetTargetName(target), "arc", 3))
  3788. /* Emit to assembly file instead for arc target
  3789. as it cannot emit to object file */
  3790. file_type = LLVMAssemblyFile;
  3791. if (LLVMTargetMachineEmitToFile(comp_ctx->target_machine, comp_ctx->module,
  3792. file_name, file_type, &err)
  3793. != 0) {
  3794. if (err) {
  3795. LLVMDisposeMessage(err);
  3796. err = NULL;
  3797. }
  3798. aot_set_last_error("emit elf to object file failed.");
  3799. return false;
  3800. }
  3801. return true;
  3802. }