Bläddra i källkod

Wormhole chain integration

This is a squashed commit of all the changes needed to integrate
wormhole chain into the main repo.
Chirantan Ekbote 3 år sedan
förälder
incheckning
7f4b0d1a09

+ 11 - 0
.github/workflows/build.yml

@@ -213,6 +213,17 @@ jobs:
           node-version: "16"
       - run: cd clients/js && make test
 
+  # Verify wormhole chain unit tests
+  wormchain:
+    runs-on: ubuntu-20.04
+    steps:
+      - uses: actions/checkout@v2
+      - uses: actions/setup-go@v2
+        with:
+          go-version: '1.19.0'
+      - run: curl https://get.ignite.com/cli@v0.23.0 | bash && mv ignite /usr/local/bin/
+      - run: cd wormhole_chain && make test
+
   # Run Go linters, Go tests and other outside-of-Tilt things.
   lint-and-tests:
     # The linter is slow enough that we want to run it on the self-hosted runner

+ 37 - 0
Tiltfile

@@ -51,6 +51,7 @@ config.define_bool("ci_tests", False, "Enable tests runner component")
 config.define_bool("guardiand_debug", False, "Enable dlv endpoint for guardiand")
 config.define_bool("node_metrics", False, "Enable Prometheus & Grafana for Guardian metrics")
 config.define_bool("guardiand_governor", False, "Enable chain governor in guardiand")
+config.define_bool("secondWormchain", False, "Enable a second wormchain node with different validator keys")
 
 cfg = config.parse()
 num_guardians = int(cfg.get("num", "1"))
@@ -72,6 +73,7 @@ ci_tests = cfg.get("ci_tests", ci)
 guardiand_debug = cfg.get("guardiand_debug", False)
 node_metrics = cfg.get("node_metrics", False)
 guardiand_governor = cfg.get("guardiand_governor", False)
+secondWormchain = cfg.get("secondWormchain", False)
 
 if cfg.get("manual", False):
     trigger_mode = TRIGGER_MODE_MANUAL
@@ -714,3 +716,38 @@ if near:
         labels = ["near"],
         trigger_mode = trigger_mode,
     )
+
+docker_build(
+    ref = "wormhole-chaind-image",
+    context = ".",
+    dockerfile = "./wormhole_chain/Dockerfile",
+    only = [],
+    ignore = ["./wormhole_chain/testing", "./wormhole_chain/ts-sdk", "./wormhole_chain/design", "./wormhole_chain/vue", "./wormhole_chain/build/wormhole-chaind"],
+)
+
+k8s_yaml_with_ns("wormhole_chain/validators/kubernetes/wormchain-guardian-devnet.yaml")
+
+k8s_resource(
+    "guardian-validator",
+    port_forwards = [
+        port_forward(1319, container_port = 1317, name = "REST [:1319]", host = webHost),
+        port_forward(26659, container_port = 26657, name = "TENDERMINT [:26659]", host = webHost)
+    ],
+    resource_deps = [],
+    labels = ["wormchain"],
+    trigger_mode = trigger_mode,
+)
+
+if secondWormchain:
+    k8s_yaml_with_ns("wormhole_chain/validators/kubernetes/wormchain-validator2-devnet.yaml")
+
+    k8s_resource(
+        "second-validator",
+        port_forwards = [
+            port_forward(1320, container_port = 1317, name = "REST [:1320]", host = webHost),
+            port_forward(26660, container_port = 26657, name = "TENDERMINT [:26660]", host = webHost)
+        ],
+        resource_deps = [],
+        labels = ["wormchain"],
+        trigger_mode = trigger_mode,
+    )

+ 3 - 2
clients/js/main.ts

@@ -34,6 +34,7 @@ import {
   toChainName,
   isEVMChain,
   toChainId,
+  tryNativeToHexString,
 } from "@certusone/wormhole-sdk";
 import { ethers } from "ethers";
 import { NETWORKS } from "./networks";
@@ -671,7 +672,7 @@ yargs(hideBin(process.argv))
       } else if (chain === "aptos") {
         throw Error("APTOS is not supported yet");
       } else if (chain === "wormholechain") {
-        throw Error("wormholechain is not supported yet");
+        throw Error("Wormhole Chain is not supported yet");
       } else {
         // If you get a type error here, hover over `chain`'s type and it tells you
         // which cases are not handled
@@ -711,7 +712,7 @@ function parseAddress(chain: ChainName, address: string): string {
   } else if (chain === "aptos") {
     throw Error("APTOS is not supported yet");
   } else if (chain === "wormholechain") {
-    throw Error("wormholechain is not supported yet");
+    return "0x" + tryNativeToHexString(address, chain);
   } else {
     impossible(chain);
   }

+ 2 - 1
clients/js/networks.ts

@@ -345,7 +345,8 @@ const DEVNET = {
     key: undefined,
   },
   wormholechain: {
-    rpc: undefined,
+    rpc: "http://localhost:1319",
+    chain_id: "wormholechain",
     key: undefined,
   },
   aptos: {

+ 2 - 0
cosmwasm/tools/deploy.js

@@ -218,6 +218,8 @@ const contract_registrations = {
     process.env.REGISTER_TERRA_TOKEN_BRIDGE_VAA,
     // NEAR
     process.env.REGISTER_NEAR_TOKEN_BRIDGE_VAA,
+    // Wormhole Chain
+    process.env.REGISTER_WORMCHAIN_TOKEN_BRIDGE_VAA,
   ],
 };
 

+ 1 - 2
devnet/eth-devnet.yaml

@@ -53,7 +53,7 @@ spec:
           command:
             - /bin/sh
             - -c
-            - "npm run migrate && npx truffle exec scripts/deploy_test_token.js && npm run deploy-batched-vaa-sender && npx truffle exec scripts/register_solana_chain.js && npx truffle exec scripts/register_terra_chain.js && npx truffle exec scripts/register_terra2_chain.js && npx truffle exec scripts/register_bsc_chain.js && npx truffle exec scripts/register_algo_chain.js && npx truffle exec scripts/register_near_chain.js && nc -lkp 2000 0.0.0.0"
+            - "npm run migrate && npx truffle exec scripts/deploy_test_token.js && npm run deploy-batched-vaa-sender && npx truffle exec scripts/register_solana_chain.js && npx truffle exec scripts/register_terra_chain.js && npx truffle exec scripts/register_terra2_chain.js && npx truffle exec scripts/register_bsc_chain.js && npx truffle exec scripts/register_algo_chain.js && npx truffle exec scripts/register_near_chain.js && npx truffle exec scripts/register_worm_chain.js && nc -lkp 2000 0.0.0.0"
           readinessProbe:
             periodSeconds: 1
             failureThreshold: 300
@@ -66,4 +66,3 @@ spec:
             - -c
             - "npx truffle exec mine.js"
 ---
-

+ 1 - 1
devnet/eth-devnet2.yaml

@@ -55,7 +55,7 @@ spec:
           command:
             - /bin/sh
             - -c
-            - "sed -i 's/CHAIN_ID=0x2/CHAIN_ID=0x4/g;s/EVM_CHAIN_ID=1/EVM_CHAIN_ID=1397/g' .env && npm run migrate && npx truffle exec scripts/deploy_test_token.js && npx truffle exec scripts/register_solana_chain.js && npx truffle exec scripts/register_terra_chain.js && npx truffle exec scripts/register_terra2_chain.js && npx truffle exec scripts/register_eth_chain.js && npx truffle exec scripts/register_algo_chain.js && npx truffle exec scripts/register_near_chain.js && nc -lkp 2000 0.0.0.0"
+            - "sed -i 's/CHAIN_ID=0x2/CHAIN_ID=0x4/g;s/EVM_CHAIN_ID=1/EVM_CHAIN_ID=1397/g' .env && npm run migrate && npx truffle exec scripts/deploy_test_token.js && npx truffle exec scripts/register_solana_chain.js && npx truffle exec scripts/register_terra_chain.js && npx truffle exec scripts/register_terra2_chain.js && npx truffle exec scripts/register_eth_chain.js && npx truffle exec scripts/register_algo_chain.js && npx truffle exec scripts/register_near_chain.js && npx truffle exec scripts/register_worm_chain.js && nc -lkp 2000 0.0.0.0"
           readinessProbe:
             periodSeconds: 1
             failureThreshold: 300

+ 4 - 0
devnet/node.yaml

@@ -93,6 +93,10 @@ spec:
             - ws://eth-devnet:8545
             - --neonRPC
             - ws://eth-devnet:8545
+            - --wormchainWS
+            - ws://guardian-validator:26657/websocket
+            - --wormchainLCD
+            - http://guardian-validator:1317
             # - --terraWS
             # - ws://terra-terrad:26657/websocket
             # - --terraLCD

+ 26 - 0
ethereum/scripts/register_worm_chain.js

@@ -0,0 +1,26 @@
+// run this script with truffle exec
+
+const jsonfile = require("jsonfile");
+const TokenBridge = artifacts.require("TokenBridge");
+const TokenImplementation = artifacts.require("TokenImplementation");
+const BridgeImplementationFullABI = jsonfile.readFileSync("../build/contracts/BridgeImplementation.json").abi;
+const wormchainTokenBridgeVAA = process.env.REGISTER_WORMCHAIN_TOKEN_BRIDGE_VAA
+
+module.exports = async function (callback) {
+    try {
+        const accounts = await web3.eth.getAccounts();
+        const tokenBridge = new web3.eth.Contract(BridgeImplementationFullABI, TokenBridge.address
+        );
+
+        // Register the wormhole token bridge endpoint
+        await tokenBridge.methods.registerChain("0x" + wormchainTokenBridgeVAA).send({
+            value: 0,
+            from: accounts[0],
+            gasLimit: 2000000
+        });
+
+        callback();
+    } catch (e) {
+        callback(e);
+    }
+};

+ 1 - 0
node/cmd/guardiand/adminnodes.go

@@ -114,6 +114,7 @@ func runListNodes(cmd *cobra.Command, args []string) {
 		{"Terra2", vaa.ChainIDTerra2},
 		{"Pythnet", vaa.ChainIDPythNet},
 		{"Moonbeam", vaa.ChainIDMoonbeam},
+		{"Wormchain", vaa.ChainIDWormchain},
 	}
 
 	if isTestnet {

+ 21 - 0
node/cmd/guardiand/node.go

@@ -39,6 +39,7 @@ import (
 	"github.com/certusone/wormhole/node/pkg/readiness"
 	"github.com/certusone/wormhole/node/pkg/reporter"
 	"github.com/certusone/wormhole/node/pkg/supervisor"
+	"github.com/certusone/wormhole/node/pkg/wormchain"
 	eth_common "github.com/ethereum/go-ethereum/common"
 	ethcrypto "github.com/ethereum/go-ethereum/crypto"
 	"github.com/libp2p/go-libp2p/core/crypto"
@@ -129,6 +130,9 @@ var (
 	nearRPC      *string
 	nearContract *string
 
+	wormchainWS  *string
+	wormchainLCD *string
+
 	solanaRPC *string
 
 	pythnetContract *string
@@ -247,6 +251,9 @@ func init() {
 	nearRPC = NodeCmd.Flags().String("nearRPC", "", "near RPC URL")
 	nearContract = NodeCmd.Flags().String("nearContract", "", "near contract")
 
+	wormchainWS = NodeCmd.Flags().String("wormchainWS", "", "Path to wormholechaind root for websocket connection")
+	wormchainLCD = NodeCmd.Flags().String("wormchainLCD", "", "Path to LCD service root for http calls")
+
 	solanaRPC = NodeCmd.Flags().String("solanaRPC", "", "Solana RPC URL (required")
 
 	pythnetContract = NodeCmd.Flags().String("pythnetContract", "", "Address of the PythNet program (required)")
@@ -393,6 +400,9 @@ func runNode(cmd *cobra.Command, args []string) {
 	if *nearRPC != "" {
 		readiness.RegisterComponent(common.ReadinessNearSyncing)
 	}
+	if *wormchainWS != "" {
+		readiness.RegisterComponent(common.ReadinessWormchainSyncing)
+	}
 	readiness.RegisterComponent(common.ReadinessBSCSyncing)
 	readiness.RegisterComponent(common.ReadinessPolygonSyncing)
 	readiness.RegisterComponent(common.ReadinessAvalancheSyncing)
@@ -816,6 +826,7 @@ func runNode(cmd *cobra.Command, args []string) {
 	chainObsvReqC[vaa.ChainIDEthereum] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
 	chainObsvReqC[vaa.ChainIDTerra] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
 	chainObsvReqC[vaa.ChainIDTerra2] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
+	chainObsvReqC[vaa.ChainIDWormchain] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
 	chainObsvReqC[vaa.ChainIDBSC] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
 	chainObsvReqC[vaa.ChainIDPolygon] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
 	chainObsvReqC[vaa.ChainIDAvalanche] = make(chan *gossipv1.ObservationRequest, observationRequestBufferSize)
@@ -1057,6 +1068,15 @@ func runNode(cmd *cobra.Command, args []string) {
 			}
 		}
 
+		// Start Wormchain watcher only if configured
+		if *wormchainWS != "" && *wormchainLCD != "" {
+			logger.Info("Starting Wormchain watcher")
+			if err := supervisor.Run(ctx, "wormchainwatch",
+				wormchain.NewWatcher(*wormchainWS, *wormchainLCD, lockC, setC, chainObsvReqC[vaa.ChainIDWormchain]).Run); err != nil {
+				return err
+			}
+		}
+
 		if *solanaRPC != "" {
 			if err := supervisor.Run(ctx, "solwatch-confirmed",
 				solana.NewSolanaWatcher(*solanaRPC, solAddress, lockC, nil, rpc.CommitmentConfirmed, common.ReadinessSolanaSyncing, vaa.ChainIDSolana).Run); err != nil {
@@ -1102,6 +1122,7 @@ func runNode(cmd *cobra.Command, args []string) {
 			*unsafeDevMode,
 			*devNumGuardians,
 			*ethRPC,
+			*wormchainLCD,
 			attestationEvents,
 			notifier,
 			gov,

+ 1 - 0
node/pkg/common/readiness.go

@@ -25,4 +25,5 @@ const (
 	ReadinessInjectiveSyncing  readiness.Component = "injectiveSyncing"
 	ReadinessPythNetSyncing    readiness.Component = "pythnetSyncing"
 	ReadinessArbitrumSyncing   readiness.Component = "arbitrumSyncing"
+	ReadinessWormchainSyncing  readiness.Component = "wormchainSyncing"
 )

+ 5 - 0
node/pkg/processor/processor.go

@@ -107,6 +107,8 @@ type Processor struct {
 	devnetNumGuardians uint
 	devnetEthRPC       string
 
+	wormchainLCD string
+
 	attestationEvents *reporter.AttestationEventReporter
 
 	logger *zap.Logger
@@ -148,6 +150,7 @@ func NewProcessor(
 	devnetMode bool,
 	devnetNumGuardians uint,
 	devnetEthRPC string,
+	wormchainLCD string,
 	attestationEvents *reporter.AttestationEventReporter,
 	notifier *discord.DiscordNotifier,
 	g *governor.ChainGovernor,
@@ -168,6 +171,8 @@ func NewProcessor(
 		devnetEthRPC:       devnetEthRPC,
 		db:                 db,
 
+		wormchainLCD: wormchainLCD,
+
 		attestationEvents: attestationEvents,
 
 		notifier: notifier,

+ 460 - 0
node/pkg/wormchain/watcher.go

@@ -0,0 +1,460 @@
+package wormchain
+
+import (
+	"context"
+	"encoding/base64"
+	"encoding/hex"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"strconv"
+	"strings"
+	"time"
+
+	"github.com/certusone/wormhole/node/pkg/p2p"
+	gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1"
+	"github.com/prometheus/client_golang/prometheus/promauto"
+
+	"github.com/prometheus/client_golang/prometheus"
+
+	eth_common "github.com/ethereum/go-ethereum/common"
+
+	"github.com/certusone/wormhole/node/pkg/common"
+	"github.com/certusone/wormhole/node/pkg/readiness"
+	"github.com/certusone/wormhole/node/pkg/supervisor"
+	"github.com/gorilla/websocket"
+	"github.com/tidwall/gjson"
+	"github.com/wormhole-foundation/wormhole/sdk/vaa"
+	"go.uber.org/zap"
+)
+
+type (
+	// Watcher is responsible for looking over wormchain blockchain and reporting new transactions to the core bridge
+	Watcher struct {
+		urlWS  string
+		urlLCD string
+
+		msgChan chan *common.MessagePublication
+		setChan chan *common.GuardianSet
+
+		// Incoming re-observation requests from the network. Pre-filtered to only
+		// include requests for our chainID.
+		obsvReqC chan *gossipv1.ObservationRequest
+	}
+)
+
+var (
+	wormchainConnectionErrors = promauto.NewCounterVec(
+		prometheus.CounterOpts{
+			Name: "wormhole_wormchain_connection_errors_total",
+			Help: "Total number of Wormchain connection errors",
+		}, []string{"reason"})
+	wormchainMessagesConfirmed = promauto.NewCounter(
+		prometheus.CounterOpts{
+			Name: "wormhole_wormchain_messages_confirmed_total",
+			Help: "Total number of verified wormchain messages found",
+		})
+	currentWormchainHeight = promauto.NewGauge(
+		prometheus.GaugeOpts{
+			Name: "wormhole_wormchain_current_height",
+			Help: "Current wormchain slot height (at default commitment level, not the level used for observations)",
+		})
+)
+
+type clientRequest struct {
+	JSONRPC string `json:"jsonrpc"`
+	// A String containing the name of the method to be invoked.
+	Method string `json:"method"`
+	// Object to pass as request parameter to the method.
+	Params [1]string `json:"params"`
+	// The request id. This can be of any type. It is used to match the
+	// response with the request that it is replying to.
+	ID uint64 `json:"id"`
+}
+
+// NewWatcher creates a new Wormchain event watcher
+func NewWatcher(
+	urlWS string,
+	urlLCD string,
+	lockEvents chan *common.MessagePublication,
+	setEvents chan *common.GuardianSet,
+	obsvReqC chan *gossipv1.ObservationRequest) *Watcher {
+	return &Watcher{urlWS: urlWS, urlLCD: urlLCD, msgChan: lockEvents, setChan: setEvents, obsvReqC: obsvReqC}
+}
+
+func (e *Watcher) Run(ctx context.Context) error {
+	p2p.DefaultRegistry.SetNetworkStats(vaa.ChainIDWormchain, &gossipv1.Heartbeat_Network{})
+
+	errC := make(chan error)
+	logger := supervisor.Logger(ctx)
+
+	logger.Info("connecting to websocket", zap.String("url", e.urlWS))
+
+	c, _, err := websocket.DefaultDialer.DialContext(ctx, e.urlWS, nil)
+	if err != nil {
+		p2p.DefaultRegistry.AddErrorCount(vaa.ChainIDWormchain, 1)
+		wormchainConnectionErrors.WithLabelValues("websocket_dial_error").Inc()
+		return fmt.Errorf("websocket dial failed: %w", err)
+	}
+	defer c.Close()
+
+	// Subscribe transactions which cause EventPostedMessage
+	params := [...]string{"tm.event='Tx' AND certusone.wormholechain.wormhole.EventPostedMessage.sequence EXISTS"}
+	// alternately, "tm.event='Tx' AND certusone.wormholechain.wormhole.EventPostedMessage.sequence >= 0"
+	command := &clientRequest{
+		JSONRPC: "2.0",
+		Method:  "subscribe",
+		Params:  params,
+		ID:      1,
+	}
+	err = c.WriteJSON(command)
+	if err != nil {
+		p2p.DefaultRegistry.AddErrorCount(vaa.ChainIDWormchain, 1)
+		wormchainConnectionErrors.WithLabelValues("websocket_subscription_error").Inc()
+		return fmt.Errorf("websocket subscription failed: %w", err)
+	}
+
+	// Wait for the success response
+	_, _, err = c.ReadMessage()
+	if err != nil {
+		p2p.DefaultRegistry.AddErrorCount(vaa.ChainIDWormchain, 1)
+		wormchainConnectionErrors.WithLabelValues("event_subscription_error").Inc()
+		return fmt.Errorf("event subscription failed: %w", err)
+	}
+	logger.Info("subscribed to new transaction events")
+
+	readiness.SetReady(common.ReadinessWormchainSyncing)
+
+	go func() {
+		t := time.NewTicker(5 * time.Second)
+		client := &http.Client{
+			Timeout: time.Second * 5,
+		}
+
+		for {
+			<-t.C
+
+			// Query and report height and set currentWormchainHeight
+			resp, err := client.Get(fmt.Sprintf("%s/blocks/latest", e.urlLCD))
+			if err != nil {
+				logger.Error("query latest block response error", zap.Error(err))
+				continue
+			}
+			blocksBody, err := ioutil.ReadAll(resp.Body)
+			if err != nil {
+				logger.Error("query latest block response read error", zap.Error(err))
+				errC <- err
+				resp.Body.Close()
+				continue
+			}
+			resp.Body.Close()
+
+			blockJSON := string(blocksBody)
+			latestBlock := gjson.Get(blockJSON, "block.header.height")
+			logger.Info("current Wormchain height", zap.Int64("block", latestBlock.Int()))
+			currentWormchainHeight.Set(float64(latestBlock.Int()))
+			p2p.DefaultRegistry.SetNetworkStats(vaa.ChainIDWormchain, &gossipv1.Heartbeat_Network{
+				Height: latestBlock.Int(),
+			})
+		}
+	}()
+
+	//TODO verify that this needs no changes
+	go func() {
+		for {
+			select {
+			case <-ctx.Done():
+				return
+			case r := <-e.obsvReqC:
+				if vaa.ChainID(r.ChainId) != vaa.ChainIDWormchain {
+					panic("invalid chain ID")
+				}
+
+				tx := hex.EncodeToString(r.TxHash)
+
+				logger.Info("received observation request for wormchain",
+					zap.String("tx_hash", tx))
+
+				client := &http.Client{
+					Timeout: time.Second * 5,
+				}
+
+				// Query for tx by hash
+				resp, err := client.Get(fmt.Sprintf("%s/cosmos/tx/v1beta1/txs/%s", e.urlLCD, tx))
+				if err != nil {
+					logger.Error("query tx response error", zap.Error(err))
+					continue
+				}
+				txBody, err := ioutil.ReadAll(resp.Body)
+				if err != nil {
+					logger.Error("query tx response read error", zap.Error(err))
+					resp.Body.Close()
+					continue
+				}
+				resp.Body.Close()
+
+				txJSON := string(txBody)
+
+				txHashRaw := gjson.Get(txJSON, "tx_response.txhash")
+				if !txHashRaw.Exists() {
+					logger.Error("wormchain tx does not have tx hash", zap.String("payload", txJSON))
+					continue
+				}
+				txHash := txHashRaw.String()
+
+				events := gjson.Get(txJSON, "tx_response.events")
+				if !events.Exists() {
+					logger.Error("wormchain tx has no events", zap.String("payload", txJSON))
+					continue
+				}
+
+				msgs := EventsToMessagePublications(txHash, events.Array(), logger)
+				for _, msg := range msgs {
+					e.msgChan <- msg
+					wormchainMessagesConfirmed.Inc()
+				}
+			}
+		}
+	}()
+
+	go func() {
+		defer close(errC)
+
+		for {
+			_, message, err := c.ReadMessage()
+
+			if err != nil {
+				p2p.DefaultRegistry.AddErrorCount(vaa.ChainIDWormchain, 1)
+				wormchainConnectionErrors.WithLabelValues("channel_read_error").Inc()
+				logger.Error("error reading channel", zap.Error(err))
+				errC <- err
+				return
+			}
+
+			// Received a message from the blockchain
+			json := string(message)
+			txHashRaw := gjson.Get(json, "result.events.tx\\.hash.0")
+			if !txHashRaw.Exists() {
+				logger.Warn("wormchain message does not have tx hash", zap.String("payload", json))
+				continue
+			}
+			txHash := txHashRaw.String()
+
+			events := gjson.Get(json, "result.data.value.TxResult.result.events")
+			if !events.Exists() {
+				logger.Warn("wormchain message has no events", zap.String("payload", json))
+				continue
+			}
+
+			msgs := EventsToMessagePublications(txHash, events.Array(), logger)
+			for _, msg := range msgs {
+				e.msgChan <- msg
+				wormchainMessagesConfirmed.Inc()
+			}
+		}
+	}()
+
+	select {
+	case <-ctx.Done():
+		err := c.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseNormalClosure, ""))
+		if err != nil {
+			logger.Error("error on closing socket ", zap.Error(err))
+		}
+		return ctx.Err()
+	case err := <-errC:
+		return err
+	}
+}
+
+// TODO adjust this function as needed for wormchain
+func EventsToMessagePublications(txHash string, events []gjson.Result, logger *zap.Logger) []*common.MessagePublication {
+	msgs := make([]*common.MessagePublication, 0, len(events))
+	for _, event := range events {
+		if !event.IsObject() {
+			logger.Warn("wormchain event is invalid", zap.String("tx_hash", txHash), zap.String("event", event.String()))
+			continue
+		}
+		eventType := gjson.Get(event.String(), "type")
+		if eventType.String() != "certusone.wormholechain.wormhole.EventPostedMessage" {
+			continue
+		}
+
+		attributes := gjson.Get(event.String(), "attributes")
+		if !attributes.Exists() {
+			logger.Warn("wormchain message event has no attributes", zap.String("tx_hash", txHash), zap.String("event", event.String()))
+			continue
+		}
+		mappedAttributes := map[string]string{}
+		for _, attribute := range attributes.Array() {
+			if !attribute.IsObject() {
+				logger.Warn("wormchain event attribute is invalid", zap.String("tx_hash", txHash), zap.String("attribute", attribute.String()))
+				continue
+			}
+			keyBase := gjson.Get(attribute.String(), "key")
+			if !keyBase.Exists() {
+				logger.Warn("wormchain event attribute does not have key", zap.String("tx_hash", txHash), zap.String("attribute", attribute.String()))
+				continue
+			}
+			valueBase := gjson.Get(attribute.String(), "value")
+			if !valueBase.Exists() {
+				logger.Warn("wormchain event attribute does not have value", zap.String("tx_hash", txHash), zap.String("attribute", attribute.String()))
+				continue
+			}
+
+			key, err := base64.StdEncoding.DecodeString(keyBase.String())
+			if err != nil {
+				logger.Warn("wormchain event key attribute is invalid", zap.String("tx_hash", txHash), zap.String("key", keyBase.String()))
+				continue
+			}
+			value, err := base64.StdEncoding.DecodeString(valueBase.String())
+			if err != nil {
+				logger.Warn("wormchain event value attribute is invalid", zap.String("tx_hash", txHash), zap.String("key", keyBase.String()), zap.String("value", valueBase.String()))
+				continue
+			}
+
+			if _, ok := mappedAttributes[string(key)]; ok {
+				logger.Debug("duplicate key in events", zap.String("tx_hash", txHash), zap.String("key", keyBase.String()), zap.String("value", valueBase.String()))
+				continue
+			}
+
+			mappedAttributes[string(key)] = string(value)
+		}
+
+		payload, ok := mappedAttributes["payload"]
+		if !ok {
+			logger.Error("wormhole event does not have a payload field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+			continue
+		}
+
+		emitter, ok := mappedAttributes["emitter"]
+		if !ok {
+			logger.Error("wormhole event does not have a emitter field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+			continue
+		}
+
+		// currently not logged
+		// chainId, ok := mappedAttributes["message.chain_id"]
+		// if !ok {
+		// 	logger.Error("wormhole event does not have a chain_id field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+		// 	continue
+		// }
+
+		nonce, ok := mappedAttributes["nonce"]
+		if !ok {
+			logger.Error("wormhole event does not have a nonce field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+			continue
+		}
+		sequence, ok := mappedAttributes["sequence"]
+		if !ok {
+			logger.Error("wormhole event does not have a sequence field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+			continue
+		}
+
+		//TODO This is not currently logged. Change this to read off the logs once it is added.
+		blockTime, ok := "0", true //mappedAttributes["blockTime"]
+		if !ok {
+			logger.Error("wormhole event does not have a blockTime field", zap.String("tx_hash", txHash), zap.String("attributes", attributes.String()))
+			continue
+		}
+
+		logger.Info("new message detected on wormchain",
+			zap.String("chainId", vaa.ChainIDWormchain.String()),
+			zap.String("txHash", txHash),
+			zap.String("emitter", emitter),
+			zap.String("nonce", nonce),
+			zap.String("sequence", sequence),
+			zap.String("blockTime", blockTime),
+		)
+
+		emitterAddress, err := StringToAddress(emitter)
+		if err != nil {
+			logger.Error("cannot decode emitter hex", zap.String("tx_hash", txHash), zap.String("value", emitter))
+			continue
+		}
+		txHashValue, err := StringToHash(txHash)
+		if err != nil {
+			logger.Error("cannot decode tx hash hex", zap.String("tx_hash", txHash), zap.String("value", txHash))
+			continue
+		}
+		payloadValue, err := secondDecode(payload)
+		if err != nil {
+			logger.Error("cannot decode payload", zap.String("tx_hash", txHash), zap.String("value", payload))
+			continue
+		}
+
+		blockTimeInt, err := strconv.ParseInt(blockTime, 10, 64)
+		if err != nil {
+			logger.Error("blocktime cannot be parsed as int", zap.String("tx_hash", txHash), zap.String("value", blockTime))
+			continue
+		}
+		nonceInt, err := strconv.ParseUint(nonce, 10, 32)
+		if err != nil {
+			logger.Error("nonce cannot be parsed as int", zap.String("tx_hash", txHash), zap.String("value", blockTime))
+			continue
+		}
+		sequenceInt, err := stringToUint(sequence)
+		if err != nil {
+			logger.Error("sequence cannot be parsed as int", zap.String("tx_hash", txHash), zap.String("value", sequence))
+			continue
+		}
+		messagePublication := &common.MessagePublication{
+			TxHash:           txHashValue,
+			Timestamp:        time.Unix(blockTimeInt, 0), //TODO read off emitted message
+			Nonce:            uint32(nonceInt),
+			Sequence:         sequenceInt,
+			EmitterChain:     vaa.ChainIDWormchain,
+			EmitterAddress:   emitterAddress,
+			Payload:          payloadValue,
+			ConsistencyLevel: 0, // Instant finality
+		}
+		msgs = append(msgs, messagePublication)
+	}
+	return msgs
+}
+
+// TODO this encoding comes out of the logs oddly, and probably requires a change on the chain
+// StringToAddress convert string into address
+func StringToAddress(value string) (vaa.Address, error) {
+	var address vaa.Address
+	res, err := secondDecode(value)
+	if err != nil {
+		return address, err
+	}
+	copy(address[:], res)
+	return address, nil
+}
+
+func stringToUint(value string) (uint64, error) {
+	value = strings.TrimSuffix(value, "\"")
+	value = strings.TrimPrefix(value, "\"")
+	res, err := strconv.ParseUint(value, 10, 64)
+	if err != nil {
+		return 0, err
+	}
+	return res, nil
+}
+
+func secondDecode(value string) ([]byte, error) {
+	//These string are double base64 encoded, and there is a pair of quotes which get included between the first and second encoding
+	value = strings.TrimSuffix(value, "\"")
+	value = strings.TrimPrefix(value, "\"")
+	res, err := base64.StdEncoding.DecodeString(value) //hex.DecodeString(value)
+	fmt.Println("address after second decode " + string(res))
+	if err != nil {
+		return nil, err
+	}
+
+	return res, nil
+}
+
+// StringToHash convert string into transaction hash
+func StringToHash(value string) (eth_common.Hash, error) {
+	var hash eth_common.Hash
+	//TODO base64? Is this correct? Double check against the logs
+	res, err := hex.DecodeString(value)
+	if err != nil {
+		return hash, err
+	}
+	copy(hash[:], res)
+	return hash, nil
+}

+ 118 - 0
node/pkg/wormchain/watcher_test.go

@@ -0,0 +1,118 @@
+package wormchain
+
+import (
+	"testing"
+
+	eth_common "github.com/ethereum/go-ethereum/common"
+	"github.com/stretchr/testify/assert"
+	"github.com/wormhole-foundation/wormhole/sdk/vaa"
+)
+
+func TestStringToUInt(t *testing.T) {
+	tests := []struct {
+		label     string
+		str       string
+		num       uint64
+		willError bool
+	}{
+		{label: "preceding slash", str: "\"1", num: 1, willError: false},
+		{label: "trailing slash", str: "1\"", num: 1, willError: false},
+		{label: "max range", str: "1", num: 1, willError: false},
+		{label: "max range", str: "18446744073709551615", num: 18446744073709551615, willError: false},
+		{label: "negative number", str: "-1", num: 0, willError: true},
+		{label: "max range plus one", str: "18446744073709551616", num: 0, willError: true},
+	}
+
+	for _, tc := range tests {
+		t.Run(tc.label, func(t *testing.T) {
+			num, err := stringToUint(tc.str)
+
+			if tc.willError == true {
+				assert.NotNil(t, err)
+			} else {
+				assert.Nil(t, err)
+			}
+
+			assert.Equal(t, tc.num, num)
+
+		})
+	}
+}
+
+func TestSecondDecode(t *testing.T) {
+	tests := []struct {
+		label     string
+		str       string
+		bytes     []byte
+		willError bool
+	}{
+		{label: "simple", str: "Zm9vCg==", bytes: []byte{0x66, 0x6f, 0x6f, 0xa}, willError: false},
+		{label: "corrupted", str: "XXXXXaGVsbG8=", bytes: []byte(nil), willError: true},
+		{label: "address", str: "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQ=", bytes: []byte{0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4}, willError: false},
+		{label: "address", str: "\"AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQ=\"", bytes: []byte{0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4}, willError: false},
+	}
+
+	for _, tc := range tests {
+		t.Run(tc.label, func(t *testing.T) {
+			decodedBytes, err := secondDecode(tc.str)
+
+			if tc.willError == true {
+				assert.NotNil(t, err)
+			} else {
+				assert.Nil(t, err)
+			}
+
+			assert.Equal(t, tc.bytes, decodedBytes)
+		})
+	}
+}
+
+func TestStringToAddress(t *testing.T) {
+	tests := []struct {
+		label     string
+		str       string
+		addr      vaa.Address
+		willError bool
+	}{
+		{label: "simple", str: "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQ=", addr: vaa.Address{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4}, willError: false},
+	}
+
+	for _, tc := range tests {
+		t.Run(tc.label, func(t *testing.T) {
+			addr, err := StringToAddress(tc.str)
+
+			if tc.willError == true {
+				assert.NotNil(t, err)
+			} else {
+				assert.Nil(t, err)
+			}
+
+			assert.Equal(t, tc.addr, addr)
+		})
+	}
+}
+
+func TestStringToHash(t *testing.T) {
+	tests := []struct {
+		label     string
+		str       string
+		hash      eth_common.Hash
+		willError bool
+	}{
+		{label: "simple", str: "4fae136bb1fd782fe1b5180ba735cdc83bcece3f9b7fd0e5e35300a61c8acd8f", hash: eth_common.HexToHash("4fae136bb1fd782fe1b5180ba735cdc83bcece3f9b7fd0e5e35300a61c8acd8f"), willError: false},
+	}
+
+	for _, tc := range tests {
+		t.Run(tc.label, func(t *testing.T) {
+			hash, err := StringToHash(tc.str)
+
+			if tc.willError == true {
+				assert.NotNil(t, err)
+			} else {
+				assert.Nil(t, err)
+			}
+
+			assert.Equal(t, tc.hash, hash)
+		})
+	}
+}

+ 11 - 0
scripts/devnet-consts.json

@@ -234,6 +234,17 @@
                     "decimals": 6
                 }
             }
+        },
+        "3104": {
+            "rpcUrlTilt": "http://guardian-validator:1317",
+            "rpcUrlLocal": "http://localhost:1319",
+            "rpcPort": "1319",
+            "contracts": {
+                "coreEmitterAddress": "wormhole1ap5vgur5zlgys8whugfegnn43emka567dtq0jl",
+                "coreNativeAddress": "wormhole1ap5vgur5zlgys8whugfegnn43emka567dtq0jl",
+                "tokenBridgeEmitterAddress": "wormhole1zugu6cajc4z7ue29g9wnes9a5ep9cs7yu7rn3z",
+                "tokenBridgeNativeAddress": "wormhole1zugu6cajc4z7ue29g9wnes9a5ep9cs7yu7rn3z"
+            }
         }
     },
     "gancheDefaults": [

+ 7 - 0
scripts/guardian-set-init.sh

@@ -87,6 +87,7 @@ bscTokenBridge=$(jq --raw-output '.chains."4".contracts.tokenBridgeEmitterAddres
 algoTokenBridge=$(jq --raw-output '.chains."8".contracts.tokenBridgeEmitterAddress' $addressesJson)
 nearTokenBridge=$(jq --raw-output '.chains."15".contracts.tokenBridgeEmitterAddress' $addressesJson)
 terra2TokenBridge=$(jq --raw-output '.chains."18".contracts.tokenBridgeEmitterAddress' $addressesJson)
+wormchainTokenBridge=$(jq --raw-output '.chains."3104".contracts.tokenBridgeEmitterAddress' $addressesJson)
 
 solNFTBridge=$(jq --raw-output '.chains."1".contracts.nftBridgeEmitterAddress' $addressesJson)
 ethNFTBridge=$(jq --raw-output '.chains."2".contracts.nftBridgeEmitterAddress' $addressesJson)
@@ -102,6 +103,7 @@ bscTokenBridgeVAA=$(node ./clients/js/build/main.js generate registration -m Tok
 algoTokenBridgeVAA=$(node ./clients/js/build/main.js generate registration -m TokenBridge -c algorand -a ${algoTokenBridge} -g ${guardiansPrivateCSV})
 nearTokenBridgeVAA=$(node ./clients/js/build/main.js generate registration -m TokenBridge -c near -a ${nearTokenBridge} -g ${guardiansPrivateCSV})
 terra2TokenBridgeVAA=$(node ./clients/js/build/main.js generate registration -m TokenBridge -c terra2 -a ${terra2TokenBridge} -g ${guardiansPrivateCSV})
+wormchainTokenBridgeVAA=$(node ./clients/js/build/main.js generate registration -m TokenBridge -c wormholechain -a ${wormchainTokenBridge} -g ${guardiansPrivateCSV})
 
 
 # 5) create nft bridge registration VAAs
@@ -122,6 +124,7 @@ bscTokenBridge="REGISTER_BSC_TOKEN_BRIDGE_VAA"
 algoTokenBridge="REGISTER_ALGO_TOKEN_BRIDGE_VAA"
 terra2TokenBridge="REGISTER_TERRA2_TOKEN_BRIDGE_VAA"
 nearTokenBridge="REGISTER_NEAR_TOKEN_BRIDGE_VAA"
+wormchainTokenBridge="REGISTER_WORMCHAIN_TOKEN_BRIDGE_VAA"
 
 solNFTBridge="REGISTER_SOL_NFT_BRIDGE_VAA"
 ethNFTBridge="REGISTER_ETH_NFT_BRIDGE_VAA"
@@ -172,6 +175,10 @@ upsert_env_file $envFile $nearTokenBridge $nearTokenBridgeVAA
 upsert_env_file $ethFile $nearNFTBridge $nearNFTBridgeVAA
 upsert_env_file $envFile $nearNFTBridge $nearNFTBridgeVAA
 
+# wormchain token bridge
+upsert_env_file $ethFile $wormchainTokenBridge $wormchainTokenBridgeVAA
+upsert_env_file $envFile $wormchainTokenBridge $wormchainTokenBridgeVAA
+
 # 7) copy the local .env file to the solana & terra dirs, if the script is running on the host machine
 # chain dirs will not exist if running in docker for Tilt, only if running locally. check before copying.
 # copy ethFile to ethereum

+ 6 - 5
sdk/devnet_consts.go

@@ -10,11 +10,12 @@ var KnownDevnetEmitters = buildKnownEmitters(knownDevnetTokenbridgeEmitters, kno
 // KnownDevnetTokenbridgeEmitters is a map of known tokenbridge emitters used during development.
 var KnownDevnetTokenbridgeEmitters = buildEmitterMap(knownDevnetTokenbridgeEmitters)
 var knownDevnetTokenbridgeEmitters = map[vaa.ChainID]string{
-	vaa.ChainIDSolana:   "c69a1b1a65dd336bf1df6a77afb501fc25db7fc0938cb08595a9ef473265cb4f",
-	vaa.ChainIDEthereum: "0000000000000000000000000290fb167208af455bb137780163b7b7a9a10c16",
-	vaa.ChainIDTerra:    "000000000000000000000000784999135aaa8a3ca5914468852fdddbddd8789d",
-	vaa.ChainIDBSC:      "0000000000000000000000000290fb167208af455bb137780163b7b7a9a10c16",
-	vaa.ChainIDAlgorand: "8edf5b0e108c3a1a0a4b704cc89591f2ad8d50df24e991567e640ed720a94be2",
+	vaa.ChainIDSolana:    "c69a1b1a65dd336bf1df6a77afb501fc25db7fc0938cb08595a9ef473265cb4f",
+	vaa.ChainIDEthereum:  "0000000000000000000000000290fb167208af455bb137780163b7b7a9a10c16",
+	vaa.ChainIDTerra:     "000000000000000000000000784999135aaa8a3ca5914468852fdddbddd8789d",
+	vaa.ChainIDBSC:       "0000000000000000000000000290fb167208af455bb137780163b7b7a9a10c16",
+	vaa.ChainIDAlgorand:  "8edf5b0e108c3a1a0a4b704cc89591f2ad8d50df24e991567e640ed720a94be2",
+	vaa.ChainIDWormchain: "0000000000000000000000001711cd63b2c545ee6545415d3cc0bda6425c43c4",
 }
 
 // KnownDevnetNFTBridgeEmitters is a map of known NFT emitters used during development.

+ 6 - 0
sdk/vaa/structs.go

@@ -142,6 +142,8 @@ func (c ChainID) String() string {
 		return "arbitrum"
 	case ChainIDPythNet:
 		return "pythnet"
+	case ChainIDWormchain:
+		return "wormholechain"
 	default:
 		return fmt.Sprintf("unknown chain ID: %d", c)
 	}
@@ -195,6 +197,8 @@ func ChainIDFromString(s string) (ChainID, error) {
 		return ChainIDArbitrum, nil
 	case "pythnet":
 		return ChainIDPythNet, nil
+	case "wormholechain":
+		return ChainIDWormchain, nil
 	default:
 		return ChainIDUnset, fmt.Errorf("unknown chain ID: %s", s)
 	}
@@ -244,6 +248,8 @@ const (
 	ChainIDArbitrum ChainID = 23
 	// ChainIDPythNet is the ChainID of PythNet
 	ChainIDPythNet ChainID = 26
+	//ChainIDWormchain is the ChainID of Wormholechain
+	ChainIDWormchain ChainID = 3104
 
 	// ChainIDEthereumRopsten is the ChainID of Ethereum Ropsten
 	ChainIDEthereumRopsten ChainID = 10001

+ 3 - 0
sdk/vaa/structs_test.go

@@ -67,6 +67,8 @@ func TestChainIDFromString(t *testing.T) {
 		{input: "Injective", output: ChainIDInjective},
 		{input: "Arbitrum", output: ChainIDArbitrum},
 		{input: "Ethereum-ropsten", output: ChainIDEthereumRopsten},
+		{input: "Wormholechain", output: ChainIDWormchain},
+		{input: "wormholechain", output: ChainIDWormchain},
 	}
 
 	// Negative Test Cases
@@ -159,6 +161,7 @@ func TestChainId_String(t *testing.T) {
 		{input: 19, output: "injective"},
 		{input: 23, output: "arbitrum"},
 		{input: 10001, output: "ethereum-ropsten"},
+		{input: 3104, output: "wormholechain"},
 	}
 
 	for _, tc := range tests {

+ 1 - 0
solana/devnet_setup.sh

@@ -102,6 +102,7 @@ node build/main.js submit -c solana -n devnet "$REGISTER_BSC_TOKEN_BRIDGE_VAA"
 node build/main.js submit -c solana -n devnet "$REGISTER_ALGO_TOKEN_BRIDGE_VAA"
 node build/main.js submit -c solana -n devnet "$REGISTER_TERRA2_TOKEN_BRIDGE_VAA"
 node build/main.js submit -c solana -n devnet "$REGISTER_NEAR_TOKEN_BRIDGE_VAA"
+node build/main.js submit -c solana -n devnet "$REGISTER_WORMCHAIN_TOKEN_BRIDGE_VAA"
 # Register the NFT Bridge Endpoint on ETH
 node build/main.js submit -c solana -n devnet "$REGISTER_ETH_NFT_BRIDGE_VAA"
 node build/main.js submit -c solana -n devnet "$REGISTER_TERRA_NFT_BRIDGE_VAA"

+ 2 - 0
terra/tools/deploy.js

@@ -270,6 +270,8 @@ const contract_registrations = {
     process.env.REGISTER_TERRA2_TOKEN_BRIDGE_VAA,
     // NEAR
     process.env.REGISTER_NEAR_TOKEN_BRIDGE_VAA,
+    // Wormhole Chain
+    process.env.REGISTER_WORMCHAIN_TOKEN_BRIDGE_VAA,
   ],
   "nft_bridge.wasm": [
     // Solana