Refactor jail part 2 (#401)

Refactor jail so that it's more self-descriptive and easier to understand by newcomers. Also, the test coverage has been improved.

Changes requiring status-react team actions:
* Replace Parse calls with new CreateAndInitCell and ExecuteJS bindings,
* Make sure web3.isConnected is ok as its response change to boolean value.
This commit is contained in:
Adam Babik 2017-11-07 18:36:42 +01:00 committed by Ivan Tomilov
parent cb5ccb52c4
commit 086747a695
23 changed files with 1079 additions and 641 deletions

View File

@ -145,7 +145,7 @@ func (s *APITestSuite) TestCellsRemovedAfterSwitchAccount() {
require.NoError(err)
for i := 0; i < itersCount; i++ {
_, e := s.api.JailManager().NewCell(getChatId(i))
_, e := s.api.JailManager().CreateCell(getChatId(i))
require.NoError(e)
}
@ -178,7 +178,7 @@ func (s *APITestSuite) TestLogoutRemovesCells() {
err = s.api.SelectAccount(address1, TestConfig.Account1.Password)
require.NoError(err)
s.api.JailManager().Parse(testChatID, ``)
s.api.JailManager().CreateAndInitCell(testChatID)
err = s.api.Logout()
require.NoError(err)

View File

@ -42,8 +42,8 @@ func (s *JailRPCTestSuite) TestJailRPCSend() {
EnsureNodeSync(s.Backend.NodeManager())
// load Status JS and add test command to it
s.jail.BaseJS(baseStatusJSCode)
s.jail.Parse(testChatID, ``)
s.jail.SetBaseJS(baseStatusJSCode)
s.jail.CreateAndInitCell(testChatID)
// obtain VM for a given chat (to send custom JS to jailed version of Send())
cell, err := s.jail.Cell(testChatID)
@ -72,7 +72,7 @@ func (s *JailRPCTestSuite) TestIsConnected() {
s.StartTestBackend()
defer s.StopTestBackend()
s.jail.Parse(testChatID, "")
s.jail.CreateAndInitCell(testChatID)
// obtain VM for a given chat (to send custom JS to jailed version of Send())
cell, err := s.jail.Cell(testChatID)
@ -87,11 +87,9 @@ func (s *JailRPCTestSuite) TestIsConnected() {
responseValue, err := cell.Get("responseValue")
s.NoError(err, "cannot obtain result of isConnected()")
response, err := responseValue.ToString()
response, err := responseValue.ToBoolean()
s.NoError(err, "cannot parse result")
expectedResponse := `{"jsonrpc":"2.0","result":true}`
s.Equal(expectedResponse, response)
s.True(response)
}
// regression test: eth_getTransactionReceipt with invalid transaction hash should return null
@ -115,7 +113,7 @@ func (s *JailRPCTestSuite) TestContractDeployment() {
EnsureNodeSync(s.Backend.NodeManager())
// obtain VM for a given chat (to send custom JS to jailed version of Send())
s.jail.Parse(testChatID, "")
s.jail.CreateAndInitCell(testChatID)
cell, err := s.jail.Cell(testChatID)
s.NoError(err)
@ -251,9 +249,9 @@ func (s *JailRPCTestSuite) TestJailVMPersistence() {
}
jail := s.Backend.JailManager()
jail.BaseJS(baseStatusJSCode)
jail.SetBaseJS(baseStatusJSCode)
parseResult := jail.Parse(testChatID, `
parseResult := jail.CreateAndInitCell(testChatID, `
var total = 0;
_status_catalog['ping'] = function(params) {
total += Number(params.amount);

View File

@ -3,12 +3,17 @@ package jail
import (
"encoding/json"
"errors"
"fmt"
"strconv"
"strings"
"sync"
"testing"
"time"
"github.com/status-im/status-go/e2e"
"github.com/status-im/status-go/geth/common"
"github.com/status-im/status-go/geth/jail"
"github.com/status-im/status-go/geth/node"
"github.com/status-im/status-go/geth/signal"
"github.com/status-im/status-go/static"
"github.com/stretchr/testify/suite"
@ -27,60 +32,83 @@ func TestJailTestSuite(t *testing.T) {
}
type JailTestSuite struct {
suite.Suite
jail common.JailManager
e2e.NodeManagerTestSuite
Jail common.JailManager
}
func (s *JailTestSuite) SetupTest() {
s.jail = jail.New(nil)
s.NotNil(s.jail)
s.NodeManager = node.NewNodeManager()
s.Jail = jail.New(s.NodeManager)
}
func (s *JailTestSuite) TestInit() {
func (s *JailTestSuite) TearDownTest() {
s.Jail.Stop()
}
func (s *JailTestSuite) TestInitWithoutBaseJS() {
errorWrapper := func(err error) string {
return `{"error":"` + err.Error() + `"}`
}
// get cell VM w/o defining cell first
cell, err := s.jail.Cell(testChatID)
cell, err := s.Jail.Cell(testChatID)
s.EqualError(err, "cell["+testChatID+"] doesn't exist")
s.EqualError(err, "cell '"+testChatID+"' not found")
s.Nil(cell)
// create VM (w/o properly initializing base JS script)
err = errors.New("ReferenceError: '_status_catalog' is not defined")
s.Equal(errorWrapper(err), s.jail.Parse(testChatID, ``))
s.Equal(errorWrapper(err), s.Jail.CreateAndInitCell(testChatID, ``))
err = errors.New("ReferenceError: 'call' is not defined")
s.Equal(errorWrapper(err), s.jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`))
s.Equal(errorWrapper(err), s.Jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`))
// get existing cell (even though we got errors, cell was still created)
cell, err = s.jail.Cell(testChatID)
cell, err = s.Jail.Cell(testChatID)
s.NoError(err)
s.NotNil(cell)
}
func (s *JailTestSuite) TestInitWithBaseJS() {
statusJS := baseStatusJSCode + `;
_status_catalog.commands["testCommand"] = function (params) {
return params.val * params.val;
};`
s.jail.BaseJS(statusJS)
s.Jail.SetBaseJS(statusJS)
// now no error should occur
response := s.jail.Parse(testChatID, ``)
response := s.Jail.CreateAndInitCell(testChatID)
expectedResponse := `{"result": {"commands":{},"responses":{}}}`
s.Equal(expectedResponse, response)
// make sure that Call succeeds even w/o running node
response = s.jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`)
response = s.Jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`)
expectedResponse = `{"result": 144}`
s.Equal(expectedResponse, response)
}
func (s *JailTestSuite) TestParse() {
// @TODO(adam): finally, this test should pass as checking existence of `_status_catalog`
// should be done in status-react.
func (s *JailTestSuite) TestCreateAndInitCellWithoutStatusCatalog() {
response := s.Jail.CreateAndInitCell(testChatID)
s.Equal(`{"error":"ReferenceError: '_status_catalog' is not defined"}`, response)
}
// @TODO(adam): remove extra JS when checking `_status_catalog` is move to status-react.
func (s *JailTestSuite) TestMultipleInitError() {
response := s.Jail.CreateAndInitCell(testChatID, `var _status_catalog = {}`)
s.Equal(`{"result": {}}`, response)
response = s.Jail.CreateAndInitCell(testChatID)
s.Equal(`{"error":"cell with id 'testChat' already exists"}`, response)
}
// @TODO(adam): remove extra JS when checking `_status_catalog` is moved to status-react.
func (s *JailTestSuite) TestCreateAndInitCellResponse() {
extraCode := `
var _status_catalog = {
foo: 'bar'
};`
response := s.jail.Parse("newChat", extraCode)
response := s.Jail.CreateAndInitCell("newChat", extraCode)
expectedResponse := `{"result": {"foo":"bar"}}`
s.Equal(expectedResponse, response)
}
@ -91,24 +119,27 @@ func (s *JailTestSuite) TestFunctionCall() {
_status_catalog.commands["testCommand"] = function (params) {
return params.val * params.val;
};`
s.jail.Parse(testChatID, statusJS)
s.Jail.CreateAndInitCell(testChatID, statusJS)
// call with wrong chat id
response := s.jail.Call("chatIDNonExistent", "", "")
expectedError := `{"error":"cell[chatIDNonExistent] doesn't exist"}`
response := s.Jail.Call("chatIDNonExistent", "", "")
expectedError := `{"error":"cell 'chatIDNonExistent' not found"}`
s.Equal(expectedError, response)
// call extraFunc()
response = s.jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`)
response = s.Jail.Call(testChatID, `["commands", "testCommand"]`, `{"val": 12}`)
expectedResponse := `{"result": 144}`
s.Equal(expectedResponse, response)
}
func (s *JailTestSuite) TestEventSignal() {
s.jail.Parse(testChatID, "")
s.StartTestNode()
defer s.StopTestNode()
s.Jail.CreateAndInitCell(testChatID)
// obtain VM for a given chat (to send custom JS to jailed version of Send())
cell, err := s.jail.Cell(testChatID)
cell, err := s.Jail.Cell(testChatID)
s.NoError(err)
testData := "foobar"
@ -154,10 +185,69 @@ func (s *JailTestSuite) TestEventSignal() {
response, err := responseValue.ToString()
s.NoError(err, "cannot parse result")
expectedResponse := `{"jsonrpc":"2.0","result":true}`
expectedResponse := `{"result":true}`
s.Equal(expectedResponse, response)
}
// TestCallResponseOrder tests exactly the problem from
// https://github.com/status-im/status-go/issues/372
func (s *JailTestSuite) TestSendSyncResponseOrder() {
s.StartTestNode()
defer s.StopTestNode()
// `testCommand` is a simple JS function. `calculateGasPrice` makes
// an implicit JSON-RPC call via `send` handler (it's a sync call).
// `web3.eth.gasPrice` is chosen to call `send` handler under the hood
// because it's a simple RPC method and does not require any params.
statusJS := baseStatusJSCode + `;
_status_catalog.commands["testCommand"] = function (params) {
return params.val * params.val;
};
_status_catalog.commands["calculateGasPrice"] = function (n) {
var gasMultiplicator = Math.pow(1.4, n).toFixed(3);
var price = 211000000000;
try {
price = web3.eth.gasPrice;
} catch (err) {}
return price * gasMultiplicator;
};
`
s.Jail.CreateAndInitCell(testChatID, statusJS)
// Concurrently call `testCommand` and `calculateGasPrice` and do some assertions.
// If the code executed in cell's VM is not thread-safe, this test will likely panic.
N := 10
errCh := make(chan error, N)
var wg sync.WaitGroup
for i := 0; i < N; i++ {
wg.Add(1)
go func(i int) {
defer wg.Done()
res := s.Jail.Call(testChatID, `["commands", "testCommand"]`, fmt.Sprintf(`{"val": %d}`, i))
if !strings.Contains(string(res), fmt.Sprintf("result\": %d", i*i)) {
errCh <- fmt.Errorf("result should be '%d', got %s", i*i, res)
}
}(i)
wg.Add(1)
go func(i int) {
defer wg.Done()
res := s.Jail.Call(testChatID, `["commands", "calculateGasPrice"]`, fmt.Sprintf(`%d`, i))
if strings.Contains(string(res), "error") {
errCh <- fmt.Errorf("result should not contain 'error', got %s", res)
}
}(i)
}
wg.Wait()
close(errCh)
for e := range errCh {
s.NoError(e)
}
}
func (s *JailTestSuite) TestJailCellsRemovedAfterStop() {
const loopLen = 5
@ -167,8 +257,8 @@ func (s *JailTestSuite) TestJailCellsRemovedAfterStop() {
require := s.Require()
for i := 0; i < loopLen; i++ {
s.jail.Parse(getTestCellID(i), "")
cell, err := s.jail.Cell(getTestCellID(i))
s.Jail.CreateAndInitCell(getTestCellID(i))
cell, err := s.Jail.Cell(getTestCellID(i))
require.NoError(err)
_, err = cell.Run(`
var counter = 1;
@ -179,10 +269,10 @@ func (s *JailTestSuite) TestJailCellsRemovedAfterStop() {
require.NoError(err)
}
s.jail.Stop()
s.Jail.Stop()
for i := 0; i < loopLen; i++ {
_, err := s.jail.Cell(getTestCellID(i))
_, err := s.Jail.Cell(getTestCellID(i))
require.Error(err, "Expected cells removing (from Jail) after stop")
}
}

View File

@ -27,37 +27,22 @@ func (s *RPCClientTestSuite) TestNewClient() {
config, err := e2e.MakeTestNodeConfig(GetNetworkID())
s.NoError(err)
nodeStarted, err := s.NodeManager.StartNode(config)
s.NoError(err)
<-nodeStarted
node, err := s.NodeManager.Node()
s.NoError(err)
// upstream disabled, local node ok
// upstream disabled
s.False(config.UpstreamConfig.Enabled)
_, err = rpc.NewClient(node, config.UpstreamConfig)
_, err = rpc.NewClient(nil, config.UpstreamConfig)
s.NoError(err)
// upstream enabled with incorrect URL, local node ok
upstreamBad := config.UpstreamConfig
upstreamBad.Enabled = true
upstreamBad.URL = "///__httphh://///incorrect_urlxxx"
_, err = rpc.NewClient(node, upstreamBad)
s.Error(err)
// upstream enabled with correct URL, local node ok
// upstream enabled with correct URL
upstreamGood := config.UpstreamConfig
upstreamGood.Enabled = true
upstreamGood.URL = "http://example.com/rpc"
_, err = rpc.NewClient(node, upstreamGood)
_, err = rpc.NewClient(nil, upstreamGood)
s.NoError(err)
// upstream disabled, local node failed (stopped)
nodeStopped, err := s.NodeManager.StopNode()
s.NoError(err)
<-nodeStopped
_, err = rpc.NewClient(node, config.UpstreamConfig)
// upstream enabled with incorrect URL
upstreamBad := config.UpstreamConfig
upstreamBad.Enabled = true
upstreamBad.URL = "///__httphh://///incorrect_urlxxx"
_, err = rpc.NewClient(nil, upstreamBad)
s.Error(err)
}

View File

@ -45,8 +45,7 @@ func (s *WhisperJailTestSuite) StartTestBackend(opts ...e2e.TestNodeOption) {
s.WhisperAPI = whisper.NewPublicWhisperAPI(s.WhisperService())
s.Jail = s.Backend.JailManager()
s.NotNil(s.Jail)
s.Jail.BaseJS(baseStatusJSCode)
s.Jail.SetBaseJS(baseStatusJSCode)
}
func (s *WhisperJailTestSuite) AddKeyPair(address, password string) (string, error) {
@ -291,7 +290,8 @@ func (s *WhisperJailTestSuite) TestJailWhisper() {
for _, tc := range testCases {
chatID := crypto.Keccak256Hash([]byte(tc.name)).Hex()
s.Jail.Parse(chatID, makeTopicCode)
s.Jail.CreateAndInitCell(chatID, makeTopicCode)
cell, err := s.Jail.Cell(chatID)
s.NoError(err, "cannot get VM")

View File

@ -179,10 +179,10 @@ func (api *StatusAPI) DiscardTransactions(ids []common.QueuedTxID) map[common.Qu
return api.b.txQueueManager.DiscardTransactions(ids)
}
// JailParse creates a new jail cell context, with the given chatID as identifier.
// CreateAndInitCell creates a new jail cell context, with the given chatID as identifier.
// New context executes provided JavaScript code, right after the initialization.
func (api *StatusAPI) JailParse(chatID string, js string) string {
return api.b.jailManager.Parse(chatID, js)
func (api *StatusAPI) CreateAndInitCell(chatID, js string) string {
return api.b.jailManager.CreateAndInitCell(chatID, js)
}
// JailCall executes given JavaScript function w/i a jail cell context identified by the chatID.
@ -190,9 +190,14 @@ func (api *StatusAPI) JailCall(chatID, this, args string) string {
return api.b.jailManager.Call(chatID, this, args)
}
// JailBaseJS allows to setup initial JavaScript to be loaded on each jail.Parse()
func (api *StatusAPI) JailBaseJS(js string) {
api.b.jailManager.BaseJS(js)
// JailExecute allows to run arbitrary JS code within a jail cell.
func (api *StatusAPI) JailExecute(chatID, code string) string {
return api.b.jailManager.Execute(chatID, code)
}
// SetJailBaseJS allows to setup initial JavaScript to be loaded on each jail.CreateAndInitCell().
func (api *StatusAPI) SetJailBaseJS(js string) {
api.b.jailManager.SetBaseJS(js)
}
// Notify sends a push notification to the device with the given token.

View File

@ -285,26 +285,29 @@ type JailCell interface {
// Call an arbitrary JS function by name and args.
Call(item string, this interface{}, args ...interface{}) (otto.Value, error)
// Stop stops background execution of cell.
Stop()
Stop() error
}
// JailManager defines methods for managing jailed environments
type JailManager interface {
// Parse creates a new jail cell context, with the given chatID as identifier.
// New context executes provided JavaScript code, right after the initialization.
Parse(chatID, js string) string
// Call executes given JavaScript function w/i a jail cell context identified by the chatID.
Call(chatID, this, args string) string
// NewCell initializes and returns a new jail cell.
NewCell(chatID string) (JailCell, error)
// CreateCell creates a new jail cell.
CreateCell(chatID string) (JailCell, error)
// CreateAndInitCell creates a new jail cell and initialize it
// with web3 and other handlers.
CreateAndInitCell(chatID string, code ...string) string
// Cell returns an existing instance of JailCell.
Cell(chatID string) (JailCell, error)
// BaseJS allows to setup initial JavaScript to be loaded on each jail.Parse()
BaseJS(js string)
// Execute allows to run arbitrary JS code within a cell.
Execute(chatID, code string) string
// SetBaseJS allows to setup initial JavaScript to be loaded on each jail.CreateAndInitCell().
SetBaseJS(js string)
// Stop stops all background activity of jail
Stop()

View File

@ -2,6 +2,8 @@ package jail
import (
"context"
"errors"
"time"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/jail/internal/fetch"
@ -16,49 +18,68 @@ type Cell struct {
*vm.VM
id string
cancel context.CancelFunc
lo *loop.Loop
loop *loop.Loop
loopStopped chan struct{}
loopErr error
}
// newCell encapsulates what we need to create a new jailCell from the
// NewCell encapsulates what we need to create a new jailCell from the
// provided vm and eventloop instance.
func newCell(id string, ottoVM *otto.Otto) (*Cell, error) {
cellVM := vm.New(ottoVM)
func NewCell(id string) (*Cell, error) {
vm := vm.New()
lo := loop.New(vm)
lo := loop.New(cellVM)
err := registerVMHandlers(cellVM, lo)
err := registerVMHandlers(vm, lo)
if err != nil {
return nil, err
}
ctx, cancel := context.WithCancel(context.Background())
// start event loop in background
go lo.Run(ctx) //nolint: errcheck
return &Cell{
VM: cellVM,
loopStopped := make(chan struct{})
cell := Cell{
VM: vm,
id: id,
cancel: cancel,
lo: lo,
}, nil
loop: lo,
loopStopped: loopStopped,
}
// Start event loop in the background.
go func() {
err := lo.Run(ctx)
if err != context.Canceled {
cell.loopErr = err
}
close(loopStopped)
}()
return &cell, nil
}
// registerHandlers register variuous functions and handlers
// to the Otto VM, such as Fetch API callbacks or promises.
func registerVMHandlers(v *vm.VM, lo *loop.Loop) error {
func registerVMHandlers(vm *vm.VM, lo *loop.Loop) error {
// setTimeout/setInterval functions
if err := timers.Define(v, lo); err != nil {
if err := timers.Define(vm, lo); err != nil {
return err
}
// FetchAPI functions
return fetch.Define(v, lo)
return fetch.Define(vm, lo)
}
// Stop halts event loop associated with cell.
func (c *Cell) Stop() {
func (c *Cell) Stop() error {
c.cancel()
select {
case <-c.loopStopped:
return c.loopErr
case <-time.After(time.Second):
return errors.New("stopping the cell timed out")
}
}
// CallAsync puts otto's function with given args into
@ -67,7 +88,9 @@ func (c *Cell) Stop() {
// async call, like callback.
func (c *Cell) CallAsync(fn otto.Value, args ...interface{}) {
task := looptask.NewCallTask(fn, args...)
c.lo.Add(task)
// TODO(divan): review API of `loop` package, it's contrintuitive
go c.lo.Ready(task)
// Add a task to the queue.
c.loop.Add(task)
// And run the task immediately.
// It's a blocking operation.
c.loop.Ready(task)
}

View File

@ -1,4 +1,4 @@
package jail_test
package jail
import (
"net/http"
@ -7,254 +7,92 @@ import (
"time"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/jail"
"github.com/status-im/status-go/static"
"github.com/stretchr/testify/suite"
)
const (
testChatID = "testChat"
)
var (
baseStatusJSCode = string(static.MustAsset("testdata/jail/status.js"))
)
func TestCellTestSuite(t *testing.T) {
suite.Run(t, new(CellTestSuite))
}
type CellTestSuite struct {
suite.Suite
jail *jail.Jail
cell *Cell
}
func (s *CellTestSuite) SetupTest() {
s.jail = jail.New(nil)
s.NotNil(s.jail)
cell, err := NewCell("testCell1")
s.NoError(err)
s.NotNil(cell)
s.cell = cell
}
func (s *CellTestSuite) TestJailTimeout() {
require := s.Require()
cell, err := s.jail.NewCell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
// Attempt to run a timeout string against a Cell.
_, err = cell.Run(`
var timerCounts = 0;
setTimeout(function(n){
if (Date.now() - n < 50) {
throw new Error("Timed out");
}
timerCounts++;
}, 50, Date.now());
`)
require.NoError(err)
// wait at least 10x longer to decrease probability
// of false negatives as we using real clock here
time.Sleep(300 * time.Millisecond)
value, err := cell.Get("timerCounts")
require.NoError(err)
require.True(value.IsNumber())
require.Equal("1", value.String())
func (s *CellTestSuite) TearDownTest() {
err := s.cell.Stop()
s.NoError(err)
}
func (s *CellTestSuite) TestJailLoopInCall() {
require := s.Require()
func (s *CellTestSuite) TestCellRegisteredHandlers() {
_, err := s.cell.Run(`setTimeout(function(){}, 100)`)
s.NoError(err)
// load Status JS and add test command to it
s.jail.BaseJS(baseStatusJSCode)
s.jail.Parse(testChatID, ``)
cell, err := s.jail.Cell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
items := make(chan string)
err = cell.Set("__captureResponse", func(val string) otto.Value {
go func() { items <- val }()
return otto.UndefinedValue()
})
require.NoError(err)
_, err = cell.Run(`
function callRunner(namespace){
console.log("Initiating callRunner for: ", namespace)
return setTimeout(function(){
__captureResponse(namespace);
}, 1000);
}
`)
require.NoError(err)
_, err = cell.Call("callRunner", nil, "softball")
require.NoError(err)
select {
case received := <-items:
require.Equal(received, "softball")
case <-time.After(5 * time.Second):
require.Fail("Failed to received event response")
}
_, err = s.cell.Run(`fetch`)
s.NoError(err)
}
// TestJailLoopRace tests multiple setTimeout callbacks,
// supposed to be run with '-race' flag.
func (s *CellTestSuite) TestJailLoopRace() {
require := s.Require()
cell, err := s.jail.NewCell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
func (s *CellTestSuite) TestCellLoopRace() {
cell := s.cell
items := make(chan struct{})
err = cell.Set("__captureResponse", func() otto.Value {
go func() { items <- struct{}{} }()
err := cell.Set("__captureResponse", func() otto.Value {
items <- struct{}{}
return otto.UndefinedValue()
})
require.NoError(err)
s.NoError(err)
_, err = cell.Run(`
function callRunner(){
return setTimeout(function(){
__captureResponse();
}, 1000);
}, 200);
}
`)
require.NoError(err)
s.NoError(err)
for i := 0; i < 100; i++ {
_, err = cell.Call("callRunner", nil)
require.NoError(err)
s.NoError(err)
}
for i := 0; i < 100; i++ {
select {
case <-items:
case <-time.After(5 * time.Second):
require.Fail("test timed out")
case <-time.After(400 * time.Millisecond):
s.Fail("test timed out")
}
}
}
func (s *CellTestSuite) TestJailFetchPromise() {
body := `{"key": "value"}`
server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
w.Header().Add("Content-Type", "application/json")
w.Write([]byte(body)) //nolint: errcheck
}))
defer server.Close()
require := s.Require()
cell, err := s.jail.NewCell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
dataCh := make(chan otto.Value, 1)
errCh := make(chan otto.Value, 1)
err = cell.Set("__captureSuccess", func(res otto.Value) { dataCh <- res })
require.NoError(err)
err = cell.Set("__captureError", func(res otto.Value) { errCh <- res })
require.NoError(err)
// run JS code for fetching valid URL
_, err = cell.Run(`fetch('` + server.URL + `').then(function(r) {
return r.text()
}).then(function(data) {
__captureSuccess(data)
}).catch(function (e) {
__captureError(e)
})`)
require.NoError(err)
select {
case data := <-dataCh:
require.True(data.IsString())
require.Equal(body, data.String())
case err := <-errCh:
require.Fail("request failed", err)
case <-time.After(1 * time.Second):
require.Fail("test timed out")
}
}
func (s *CellTestSuite) TestJailFetchCatch() {
require := s.Require()
cell, err := s.jail.NewCell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
dataCh := make(chan otto.Value, 1)
errCh := make(chan otto.Value, 1)
err = cell.Set("__captureSuccess", func(res otto.Value) { dataCh <- res })
require.NoError(err)
err = cell.Set("__captureError", func(res otto.Value) { errCh <- res })
require.NoError(err)
// run JS code for fetching invalid URL
_, err = cell.Run(`fetch('http://👽/nonexistent').then(function(r) {
return r.text()
}).then(function(data) {
__captureSuccess(data)
}).catch(function (e) {
__captureError(e)
})`)
require.NoError(err)
select {
case data := <-dataCh:
require.Fail("request should have failed, but returned", data)
case e := <-errCh:
require.True(e.IsObject())
name, err := e.Object().Get("name")
require.NoError(err)
require.Equal("Error", name.String())
_, err = e.Object().Get("message")
require.NoError(err)
case <-time.After(3 * time.Second):
require.Fail("test timed out")
}
}
// TestJailFetchRace tests multiple fetch callbacks,
// supposed to be run with '-race' flag.
func (s *CellTestSuite) TestJailFetchRace() {
func (s *CellTestSuite) TestCellFetchRace() {
body := `{"key": "value"}`
server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
w.Header().Add("Content-Type", "application/json")
w.Write([]byte(body)) //nolint: errcheck
}))
defer server.Close()
require := s.Require()
cell, err := s.jail.NewCell(testChatID)
require.NoError(err)
require.NotNil(cell)
defer cell.Stop()
cell := s.cell
dataCh := make(chan otto.Value, 1)
errCh := make(chan otto.Value, 1)
err = cell.Set("__captureSuccess", func(res otto.Value) { dataCh <- res })
require.NoError(err)
err := cell.Set("__captureSuccess", func(res otto.Value) { dataCh <- res })
s.NoError(err)
err = cell.Set("__captureError", func(res otto.Value) { errCh <- res })
require.NoError(err)
s.NoError(err)
// run JS code for fetching valid URL
_, err = cell.Run(`fetch('` + server.URL + `').then(function(r) {
@ -264,7 +102,7 @@ func (s *CellTestSuite) TestJailFetchRace() {
}).catch(function (e) {
__captureError(e)
})`)
require.NoError(err)
s.NoError(err)
// run JS code for fetching invalid URL
_, err = cell.Run(`fetch('http://👽/nonexistent').then(function(r) {
@ -274,74 +112,93 @@ func (s *CellTestSuite) TestJailFetchRace() {
}).catch(function (e) {
__captureError(e)
})`)
require.NoError(err)
s.NoError(err)
for i := 0; i < 2; i++ {
select {
case data := <-dataCh:
require.True(data.IsString())
require.Equal(body, data.String())
s.Equal(body, data.String())
case e := <-errCh:
require.True(e.IsObject())
name, err := e.Object().Get("name")
require.NoError(err)
require.Equal("Error", name.String())
s.NoError(err)
s.Equal("Error", name.String())
_, err = e.Object().Get("message")
require.NoError(err)
case <-time.After(3 * time.Second):
require.Fail("test timed out")
s.NoError(err)
case <-time.After(5 * time.Second):
s.Fail("test timed out")
return
}
}
}
// TestJailLoopCancel tests that cell.Stop() really cancels event
// TestCellLoopCancel tests that cell.Stop() really cancels event
// loop and pending tasks.
func (s *CellTestSuite) TestJailLoopCancel() {
require := s.Require()
// load Status JS and add test command to it
s.jail.BaseJS(baseStatusJSCode)
s.jail.Parse(testChatID, ``)
cell, err := s.jail.Cell(testChatID)
require.NoError(err)
require.NotNil(cell)
func (s *CellTestSuite) TestCellLoopCancel() {
cell := s.cell
var err error
var count int
err = cell.Set("__captureResponse", func(val string) otto.Value { //nolint: unparam
err = cell.Set("__captureResponse", func(call otto.FunctionCall) otto.Value {
count++
return otto.UndefinedValue()
})
require.NoError(err)
s.NoError(err)
_, err = cell.Run(`
function callRunner(val, delay){
function callRunner(delay){
return setTimeout(function(){
__captureResponse(val);
__captureResponse();
}, delay);
}
`)
require.NoError(err)
s.NoError(err)
// Run 5 timeout tasks to be executed in: 1, 2, 3, 4 and 5 secs
for i := 1; i <= 5; i++ {
_, err = cell.Call("callRunner", nil, "value", i*1000)
require.NoError(err)
_, err = cell.Call("callRunner", nil, i*1000)
s.NoError(err)
}
// Wait 1.5 second (so only one task executed) so far
// and stop the cell (event loop should die)
time.Sleep(1500 * time.Millisecond)
cell.Stop()
err = cell.Stop()
s.NoError(err)
// check that only 1 task has increased counter
require.Equal(1, count)
s.Equal(1, count)
// wait 2 seconds more (so at least two more tasks would
// have been executed if event loop is still running)
<-time.After(2 * time.Second)
// check that counter hasn't increased
require.Equal(1, count)
s.Equal(1, count)
}
func (s *CellTestSuite) TestCellCallAsync() {
// Don't use buffered channel as it's supposed to be an async call.
datac := make(chan string)
err := s.cell.Set("testCallAsync", func(call otto.FunctionCall) otto.Value {
datac <- call.Argument(0).String()
return otto.UndefinedValue()
})
s.NoError(err)
fn, err := s.cell.Get("testCallAsync")
s.NoError(err)
s.cell.CallAsync(fn, "success")
s.Equal("success", <-datac)
}
func (s *CellTestSuite) TestCellCallStopMultipleTimes() {
s.NotPanics(func() {
err := s.cell.Stop()
s.NoError(err)
err = s.cell.Stop()
s.NoError(err)
})
}

View File

@ -4,130 +4,179 @@ import (
"os"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/common"
"github.com/status-im/status-go/geth/jail/console"
"github.com/status-im/status-go/geth/node"
"github.com/status-im/status-go/geth/signal"
)
// signals
const (
// EventSignal is a signal from jail.
EventSignal = "jail.signal"
// EventConsoleLog defines the event type for the console.log call.
// eventConsoleLog defines the event type for the console.log call.
eventConsoleLog = "vm.console.log"
)
// registerHandlers augments and transforms a given jail cell's underlying VM,
// by adding and replacing method handlers.
func registerHandlers(jail *Jail, cell common.JailCell, chatID string) error {
jeth, err := cell.Get("jeth")
if err != nil {
return err
}
registerHandler := jeth.Object().Set
if err = registerHandler("console", map[string]interface{}{
// registerWeb3Provider creates an object called "jeth",
// which is a web3.js provider.
func registerWeb3Provider(jail *Jail, cell *Cell) error {
jeth := map[string]interface{}{
"console": map[string]interface{}{
"log": func(fn otto.FunctionCall) otto.Value {
return console.Write(fn, os.Stdout, eventConsoleLog)
},
}); err != nil {
return err
},
"send": createSendHandler(jail, cell),
"sendAsync": createSendAsyncHandler(jail, cell),
"isConnected": createIsConnectedHandler(jail),
}
// register send handler
if err = registerHandler("send", makeSendHandler(jail)); err != nil {
return err
}
// register sendAsync handler
if err = registerHandler("sendAsync", makeAsyncSendHandler(jail, cell)); err != nil {
return err
}
// register isConnected handler
if err = registerHandler("isConnected", makeJethIsConnectedHandler(jail, cell)); err != nil {
return err
}
// register sendMessage/showSuggestions handlers
if err = cell.Set("statusSignals", struct{}{}); err != nil {
return err
}
statusSignals, err := cell.Get("statusSignals")
if err != nil {
return err
}
registerHandler = statusSignals.Object().Set
return registerHandler("sendSignal", makeSignalHandler(chatID))
return cell.Set("jeth", jeth)
}
// makeAsyncSendHandler returns jeth.sendAsync() handler.
func makeAsyncSendHandler(jail *Jail, cellInt common.JailCell) func(call otto.FunctionCall) otto.Value {
// FIXME(tiabc): Get rid of this.
cell := cellInt.(*Cell)
return func(call otto.FunctionCall) otto.Value {
go func() {
response := jail.Send(call)
// registerStatusSignals creates an object called "statusSignals".
// TODO(adam): describe what it is and when it's used.
func registerStatusSignals(cell *Cell) error {
statusSignals := map[string]interface{}{
"sendSignal": createSendSignalHandler(cell),
}
return cell.Set("statusSignals", statusSignals)
}
// createSendHandler returns jeth.send().
func createSendHandler(jail *Jail, cell *Cell) func(call otto.FunctionCall) otto.Value {
return func(call otto.FunctionCall) otto.Value {
// As it's a sync call, it's called already from a thread-safe context,
// thus using otto.Otto directly. Otherwise, it would try to acquire a lock again
// and result in a deadlock.
vm := cell.VM.UnsafeVM()
request, err := vm.Call("JSON.stringify", nil, call.Argument(0))
if err != nil {
throwJSError(err)
}
response, err := jail.sendRPCCall(request.String())
if err != nil {
throwJSError(err)
}
value, err := vm.ToValue(response)
if err != nil {
throwJSError(err)
}
return value
}
}
// createSendAsyncHandler returns jeth.sendAsync() handler.
func createSendAsyncHandler(jail *Jail, cell *Cell) func(call otto.FunctionCall) otto.Value {
return func(call otto.FunctionCall) otto.Value {
// As it's a sync call, it's called already from a thread-safe context,
// thus using otto.Otto directly. Otherwise, it would try to acquire a lock again
// and result in a deadlock.
unsafeVM := cell.VM.UnsafeVM()
request, err := unsafeVM.Call("JSON.stringify", nil, call.Argument(0))
if err != nil {
throwJSError(err)
}
go func() {
// As it's an async call, it's not called from a thread-safe context,
// thus using a thread-safe vm.VM.
vm := cell.VM
callback := call.Argument(1)
if callback.Class() == "Function" {
// run callback asyncronously with args (error, response)
err := otto.NullValue()
cell.CallAsync(callback, err, response)
response, err := jail.sendRPCCall(request.String())
// If provided callback argument is not a function, don't call it.
if callback.Class() != "Function" {
return
}
if err != nil {
cell.CallAsync(callback, vm.MakeCustomError("Error", err.Error()))
} else {
cell.CallAsync(callback, nil, response)
}
}()
return otto.UndefinedValue()
}
}
// makeSendHandler returns jeth.send() and jeth.sendAsync() handler
func makeSendHandler(jail *Jail) func(call otto.FunctionCall) otto.Value {
// createIsConnectedHandler returns jeth.isConnected() handler.
// This handler returns `true` if client is actively listening for network connections.
func createIsConnectedHandler(jail RPCClientProvider) func(call otto.FunctionCall) otto.Value {
return func(call otto.FunctionCall) otto.Value {
return jail.Send(call)
client := jail.RPCClient()
if client == nil {
throwJSError(ErrNoRPCClient)
}
}
// makeJethIsConnectedHandler returns jeth.isConnected() handler
func makeJethIsConnectedHandler(jail *Jail, cellInt common.JailCell) func(call otto.FunctionCall) (response otto.Value) {
// FIXME(tiabc): Get rid of this.
cell := cellInt.(*Cell)
return func(call otto.FunctionCall) otto.Value {
client := jail.nodeManager.RPCClient()
var netListeningResult bool
if err := client.Call(&netListeningResult, "net_listening"); err != nil {
return newErrorResponseOtto(cell.VM, err.Error(), nil)
throwJSError(err)
}
if !netListeningResult {
return newErrorResponseOtto(cell.VM, node.ErrNoRunningNode.Error(), nil)
if netListeningResult {
return otto.TrueValue()
}
return newResultResponse(call.Otto, true)
return otto.FalseValue()
}
}
// SignalEvent wraps Jail send signals
type SignalEvent struct {
ChatID string `json:"chat_id"`
Data string `json:"data"`
}
func makeSignalHandler(chatID string) func(call otto.FunctionCall) otto.Value {
func createSendSignalHandler(cell *Cell) func(otto.FunctionCall) otto.Value {
return func(call otto.FunctionCall) otto.Value {
message := call.Argument(0).String()
signal.Send(signal.Envelope{
Type: EventSignal,
Event: SignalEvent{
ChatID: chatID,
Event: struct {
ChatID string `json:"chat_id"`
Data string `json:"data"`
}{
ChatID: cell.id,
Data: message,
},
})
return newResultResponse(call.Otto, true)
// As it's a sync call, it's called already from a thread-safe context,
// thus using otto.Otto directly. Otherwise, it would try to acquire a lock again
// and result in a deadlock.
vm := cell.VM.UnsafeVM()
value, err := wrapResultInValue(vm, otto.TrueValue())
if err != nil {
throwJSError(err)
}
return value
}
}
// throwJSError calls panic with an error string. It should be called
// only in a context that handles panics like otto.Otto.
func throwJSError(err error) {
value, err := otto.ToValue(err.Error())
if err != nil {
panic(err.Error())
}
panic(value)
}
func wrapResultInValue(vm *otto.Otto, result interface{}) (value otto.Value, err error) {
value, err = vm.Run(`({})`)
if err != nil {
return
}
err = value.Object().Set("result", result)
if err != nil {
return
}
return
}

186
geth/jail/handlers_test.go Normal file
View File

@ -0,0 +1,186 @@
package jail
import (
"fmt"
"net/http"
"net/http/httptest"
"testing"
"time"
"github.com/robertkrimen/otto"
gethrpc "github.com/ethereum/go-ethereum/rpc"
"github.com/status-im/status-go/geth/params"
"github.com/status-im/status-go/geth/rpc"
"github.com/status-im/status-go/geth/signal"
"github.com/stretchr/testify/suite"
)
func TestHandlersTestSuite(t *testing.T) {
suite.Run(t, new(HandlersTestSuite))
}
type HandlersTestSuite struct {
suite.Suite
responseFixture string
ts *httptest.Server
tsCalls int
client *gethrpc.Client
}
func (s *HandlersTestSuite) SetupTest() {
s.responseFixture = `{"json-rpc":"2.0","id":10,"result":true}`
s.ts = httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
s.tsCalls++
fmt.Fprintln(w, s.responseFixture)
}))
client, err := gethrpc.Dial(s.ts.URL)
s.NoError(err)
s.client = client
}
func (s *HandlersTestSuite) TearDownTest() {
s.ts.Close()
s.tsCalls = 0
}
func (s *HandlersTestSuite) TestWeb3SendHandlerSuccess() {
client, err := rpc.NewClient(s.client, params.UpstreamRPCConfig{})
s.NoError(err)
jail := New(&testRPCClientProvider{client})
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
// web3.eth.syncing is an arbitrary web3 sync RPC call.
value, err := cell.Run("web3.eth.syncing")
s.NoError(err)
result, err := value.ToBoolean()
s.NoError(err)
s.True(result)
}
func (s *HandlersTestSuite) TestWeb3SendHandlerFailure() {
jail := New(nil)
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
_, err = cell.Run("web3.eth.syncing")
s.Error(err, ErrNoRPCClient.Error())
}
func (s *HandlersTestSuite) TestWeb3SendAsyncHandlerSuccess() {
client, err := rpc.NewClient(s.client, params.UpstreamRPCConfig{})
s.NoError(err)
jail := New(&testRPCClientProvider{client})
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
errc := make(chan string)
resultc := make(chan string)
err = cell.Set("__getSyncingCallback", func(call otto.FunctionCall) otto.Value {
errc <- call.Argument(0).String()
resultc <- call.Argument(1).String()
return otto.UndefinedValue()
})
s.NoError(err)
_, err = cell.Run(`web3.eth.getSyncing(__getSyncingCallback)`)
s.NoError(err)
s.Equal(`null`, <-errc)
s.Equal(`true`, <-resultc)
}
func (s *HandlersTestSuite) TestWeb3SendAsyncHandlerWithoutCallbackSuccess() {
client, err := rpc.NewClient(s.client, params.UpstreamRPCConfig{})
s.NoError(err)
jail := New(&testRPCClientProvider{client})
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
_, err = cell.Run(`web3.eth.getSyncing()`)
s.NoError(err)
// As there is no callback, it's not possible to detect when
// the request hit the server.
time.Sleep(time.Millisecond * 100)
s.Equal(1, s.tsCalls)
}
func (s *HandlersTestSuite) TestWeb3SendAsyncHandlerFailure() {
jail := New(nil)
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
errc := make(chan otto.Value)
resultc := make(chan string)
err = cell.Set("__getSyncingCallback", func(call otto.FunctionCall) otto.Value {
errc <- call.Argument(0)
resultc <- call.Argument(1).String()
return otto.UndefinedValue()
})
s.NoError(err)
_, err = cell.Run(`web3.eth.getSyncing(__getSyncingCallback)`)
s.NoError(err)
errValue := <-errc
message, err := errValue.Object().Get("message")
s.NoError(err)
s.Equal(ErrNoRPCClient.Error(), message.String())
s.Equal(`undefined`, <-resultc)
}
func (s *HandlersTestSuite) TestWeb3IsConnectedHandler() {
client, err := rpc.NewClient(s.client, params.UpstreamRPCConfig{})
s.NoError(err)
jail := New(&testRPCClientProvider{client})
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
// When result is true.
value, err := cell.Run("web3.isConnected()")
s.NoError(err)
valueBoolean, err := value.ToBoolean()
s.NoError(err)
s.True(valueBoolean)
// When result is false.
s.responseFixture = `{"json-rpc":"2.0","id":10,"result":false}`
value, err = cell.Run("web3.isConnected()")
s.NoError(err)
valueBoolean, err = value.ToBoolean()
s.NoError(err)
s.False(valueBoolean)
}
func (s *HandlersTestSuite) TestSendSignalHandler() {
jail := New(nil)
cell, err := jail.createAndInitCell("cell1")
s.NoError(err)
signal.SetDefaultNodeNotificationHandler(func(jsonEvent string) {
s.Contains(jsonEvent, "test signal message")
})
value, err := cell.Run(`statusSignals.sendSignal("test signal message")`)
s.NoError(err)
result, err := value.Object().Get("result")
s.NoError(err)
resultBool, err := result.ToBoolean()
s.NoError(err)
s.True(resultBool)
}

View File

@ -207,8 +207,7 @@ func (s *FetchSuite) SetupTest() {
s.mux = http.NewServeMux()
s.srv = httptest.NewServer(s.mux)
o := otto.New()
s.vm = vm.New(o)
s.vm = vm.New()
s.loop = loop.New(s.vm)
go s.loop.Run(context.Background()) //nolint: errcheck

View File

@ -90,6 +90,15 @@ func (l *Loop) removeByID(id int64) {
l.lock.Unlock()
}
func (l *Loop) removeAll() {
l.lock.Lock()
for _, t := range l.tasks {
t.Cancel()
}
l.tasks = make(map[int64]Task)
l.lock.Unlock()
}
// Ready signals to the loop that a task is ready to be finalised. This might
// block if the "ready channel" in the loop is at capacity.
func (l *Loop) Ready(t Task) {
@ -108,9 +117,7 @@ func (l *Loop) processTask(t Task) error {
if err := t.Execute(l.vm, l); err != nil {
l.lock.RLock()
for _, t := range l.tasks {
t.Cancel()
}
l.lock.RUnlock()
return err
@ -127,6 +134,11 @@ func (l *Loop) Run(ctx context.Context) error {
for {
select {
case t := <-l.ready:
if ctx.Err() != nil {
l.removeAll()
return ctx.Err()
}
if t == nil {
continue
}
@ -140,7 +152,8 @@ func (l *Loop) Run(ctx context.Context) error {
continue
}
case <-ctx.Done():
return context.Canceled
l.removeAll()
return ctx.Err()
}
}
}

View File

@ -121,9 +121,9 @@ func (c CallTask) Execute(vm *vm.VM, l *loop.Loop) error {
// FunctionCall in CallTask likely does use it,
// so we must to guard it here
vm.Lock()
defer vm.Unlock()
v, err := c.Function.Call(otto.NullValue(), c.Args...)
vm.Unlock()
c.Value <- v
c.Error <- err

View File

@ -5,7 +5,6 @@ import (
"testing"
"time"
"github.com/robertkrimen/otto"
"github.com/stretchr/testify/suite"
"github.com/status-im/status-go/geth/jail/internal/loop"
@ -85,8 +84,7 @@ type PromiseSuite struct {
}
func (s *PromiseSuite) SetupTest() {
o := otto.New()
s.vm = vm.New(o)
s.vm = vm.New()
s.loop = loop.New(s.vm)
go s.loop.Run(context.Background()) //nolint: errcheck

View File

@ -5,8 +5,6 @@ import (
"testing"
"time"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/jail/internal/loop"
"github.com/status-im/status-go/geth/jail/internal/timers"
"github.com/status-im/status-go/geth/jail/internal/vm"
@ -103,8 +101,7 @@ type TimersSuite struct {
}
func (s *TimersSuite) SetupTest() {
o := otto.New()
s.vm = vm.New(o)
s.vm = vm.New()
s.loop = loop.New(s.vm)
go s.loop.Run(context.Background()) //nolint: errcheck

View File

@ -15,12 +15,17 @@ type VM struct {
}
// New creates new instance of VM.
func New(vm *otto.Otto) *VM {
func New() *VM {
return &VM{
vm: vm,
vm: otto.New(),
}
}
// UnsafeVM returns a thread-unsafe JavaScript VM.
func (vm *VM) UnsafeVM() *otto.Otto {
return vm.vm
}
// Set sets the value to be keyed by the provided keyname.
func (vm *VM) Set(key string, val interface{}) error {
vm.Lock()
@ -77,3 +82,11 @@ func (vm *VM) ToValue(value interface{}) (otto.Value, error) {
return vm.vm.ToValue(value)
}
// MakeCustomError allows to create a new Error object.
func (vm *VM) MakeCustomError(name, message string) otto.Value {
vm.Lock()
defer vm.Unlock()
return vm.vm.MakeCustomError(name, message)
}

View File

@ -4,251 +4,281 @@ import (
"encoding/json"
"errors"
"fmt"
"strings"
"sync"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/common"
"github.com/status-im/status-go/geth/jail/internal/vm"
"github.com/status-im/status-go/geth/log"
"github.com/status-im/status-go/geth/rpc"
"github.com/status-im/status-go/static"
)
var (
// FIXME(tiabc): Get rid of this global variable. Move it to a constructor or initialization.
web3JSCode = static.MustAsset("scripts/web3.js")
//ErrInvalidJail - error jail init env
ErrInvalidJail = errors.New("jail environment is not properly initialized")
const (
web3InstanceCode = `
var Web3 = require('web3');
var web3 = new Web3(jeth);
var Bignumber = require("bignumber.js");
function bn(val) {
return new Bignumber(val);
}
`
)
// Jail represents jailed environment inside of which we hold multiple cells.
// Each cell is a separate JavaScript VM.
var (
web3Code = string(static.MustAsset("scripts/web3.js"))
// ErrNoRPCClient is returned when an RPC client is required but it's nil.
ErrNoRPCClient = errors.New("RPC client is not available")
)
// RPCClientProvider is an interface that provides a way
// to obtain an rpc.Client.
type RPCClientProvider interface {
RPCClient() *rpc.Client
}
// Jail manages multiple JavaScript execution contexts (JavaScript VMs) called cells.
// Each cell is a separate VM with web3.js set up.
//
// As rpc.Client might not be available during Jail initialization,
// a provider function is used.
type Jail struct {
nodeManager common.NodeManager
baseJSCode string // JavaScript used to initialize all new cells with
rpcClientProvider RPCClientProvider
baseJS string
cellsMx sync.RWMutex
cells map[string]*Cell // jail supports running many isolated instances of jailed runtime
vm *vm.VM // vm for internal otto related tasks (see Send method)
cells map[string]*Cell
}
// New returns new Jail environment with the associated NodeManager.
// It's caller responsibility to call jail.Stop() when jail is not needed.
func New(nodeManager common.NodeManager) *Jail {
// New returns a new Jail.
func New(provider RPCClientProvider) *Jail {
return NewWithBaseJS(provider, "")
}
// NewWithBaseJS returns a new Jail with base JS configured.
func NewWithBaseJS(provider RPCClientProvider, code string) *Jail {
return &Jail{
nodeManager: nodeManager,
rpcClientProvider: provider,
baseJS: code,
cells: make(map[string]*Cell),
vm: vm.New(otto.New()),
}
}
// BaseJS allows to setup initial JavaScript to be loaded on each jail.Parse().
func (jail *Jail) BaseJS(js string) {
jail.baseJSCode = js
// SetBaseJS sets initial JavaScript code loaded to each new cell.
func (j *Jail) SetBaseJS(js string) {
j.baseJS = js
}
// NewCell initializes and returns a new jail cell.
func (jail *Jail) NewCell(chatID string) (common.JailCell, error) {
if jail == nil {
return nil, ErrInvalidJail
// Stop stops jail and all assosiacted cells.
func (j *Jail) Stop() {
j.cellsMx.Lock()
defer j.cellsMx.Unlock()
for _, cell := range j.cells {
cell.Stop() //nolint: errcheck
}
cellVM := otto.New()
// TODO(tiabc): Move this initialisation to a proper place.
j.cells = make(map[string]*Cell)
}
cell, err := newCell(chatID, cellVM)
// createCell creates a new cell if it does not exists.
func (j *Jail) createCell(chatID string) (*Cell, error) {
j.cellsMx.Lock()
defer j.cellsMx.Unlock()
if cell, ok := j.cells[chatID]; ok {
return cell, fmt.Errorf("cell with id '%s' already exists", chatID)
}
cell, err := NewCell(chatID)
if err != nil {
return nil, err
}
jail.cellsMx.Lock()
jail.cells[chatID] = cell
jail.cellsMx.Unlock()
j.cells[chatID] = cell
return cell, nil
}
// Stop stops jail and all assosiacted cells.
func (jail *Jail) Stop() {
jail.cellsMx.Lock()
defer jail.cellsMx.Unlock()
for _, cell := range jail.cells {
cell.Stop()
}
// TODO(tiabc): Move this initialisation to a proper place.
jail.cells = make(map[string]*Cell)
// CreateCell creates a new cell. It returns an error
// if a cell with a given ID already exists.
func (j *Jail) CreateCell(chatID string) (common.JailCell, error) {
return j.createCell(chatID)
}
// Cell returns the existing instance of Cell.
func (jail *Jail) Cell(chatID string) (common.JailCell, error) {
jail.cellsMx.RLock()
defer jail.cellsMx.RUnlock()
// initCell initializes a cell with default JavaScript handlers and user code.
func (j *Jail) initCell(cell *Cell) error {
// Register objects being a bridge between Go and JavaScript.
if err := registerWeb3Provider(j, cell); err != nil {
return err
}
cell, ok := jail.cells[chatID]
if err := registerStatusSignals(cell); err != nil {
return err
}
// Run some initial JS code to provide some global objects.
c := []string{
j.baseJS,
web3Code,
web3InstanceCode,
}
_, err := cell.Run(strings.Join(c, ";"))
return err
}
// CreateAndInitCell creates and initializes a new Cell.
func (j *Jail) createAndInitCell(chatID string, code ...string) (*Cell, error) {
cell, err := j.createCell(chatID)
if err != nil {
return nil, err
}
if err := j.initCell(cell); err != nil {
return nil, err
}
// Run custom user code
for _, js := range code {
_, err := cell.Run(js)
if err != nil {
return nil, err
}
}
return cell, nil
}
// CreateAndInitCell creates and initializes new Cell. Additionally,
// it creates a `catalog` variable in the VM.
// It returns the response as a JSON string.
func (j *Jail) CreateAndInitCell(chatID string, code ...string) string {
cell, err := j.createAndInitCell(chatID, code...)
if err != nil {
return newJailErrorResponse(err)
}
return j.makeCatalogVariable(cell)
}
// makeCatalogVariable provides `catalog` as a global variable.
// TODO(divan): this can and should be implemented outside of jail,
// on a clojure side. Moving this into separate method to nuke it later
// easier.
func (j *Jail) makeCatalogVariable(cell *Cell) string {
_, err := cell.Run(`var catalog = JSON.stringify(_status_catalog)`)
if err != nil {
return newJailErrorResponse(err)
}
value, err := cell.Get("catalog")
if err != nil {
return newJailErrorResponse(err)
}
return newJailResultResponse(value)
}
func (j *Jail) cell(chatID string) (*Cell, error) {
j.cellsMx.RLock()
defer j.cellsMx.RUnlock()
cell, ok := j.cells[chatID]
if !ok {
return nil, fmt.Errorf("cell[%s] doesn't exist", chatID)
return nil, fmt.Errorf("cell '%s' not found", chatID)
}
return cell, nil
}
// Parse creates a new jail cell context, with the given chatID as identifier.
// New context executes provided JavaScript code, right after the initialization.
func (jail *Jail) Parse(chatID, js string) string {
if jail == nil {
return makeError(ErrInvalidJail.Error())
}
cell, err := jail.Cell(chatID)
if err != nil {
if _, mkerr := jail.NewCell(chatID); mkerr != nil {
return makeError(mkerr.Error())
}
cell, _ = jail.Cell(chatID)
}
// init jeth and its handlers
if err = cell.Set("jeth", struct{}{}); err != nil {
return makeError(err.Error())
}
if err = registerHandlers(jail, cell, chatID); err != nil {
return makeError(err.Error())
}
initJs := jail.baseJSCode + ";"
if _, err = cell.Run(initJs); err != nil {
return makeError(err.Error())
}
jjs := string(web3JSCode) + `
var Web3 = require('web3');
var web3 = new Web3(jeth);
var Bignumber = require("bignumber.js");
function bn(val){
return new Bignumber(val);
}
` + js + "; var catalog = JSON.stringify(_status_catalog);"
if _, err = cell.Run(jjs); err != nil {
return makeError(err.Error())
}
res, err := cell.Get("catalog")
if err != nil {
return makeError(err.Error())
}
return makeResult(res.String(), err)
// Cell returns a cell by chatID. If it does not exist, error is returned.
// Required by the Backend.
func (j *Jail) Cell(chatID string) (common.JailCell, error) {
return j.cell(chatID)
}
// Call executes the `call` function w/i a jail cell context identified by the chatID.
func (jail *Jail) Call(chatID, this, args string) string {
cell, err := jail.Cell(chatID)
// Execute allows to run arbitrary JS code within a cell.
func (j *Jail) Execute(chatID, code string) string {
cell, err := j.cell(chatID)
if err != nil {
return makeError(err.Error())
return newJailErrorResponse(err)
}
res, err := cell.Call("call", nil, this, args)
return makeResult(res.String(), err)
}
// Send is a wrapper for executing RPC calls from within Otto VM.
// It uses own jail's VM instance instead of cell's one to
// increase safety of cell's vm usage.
// TODO(divan): investigate if it's possible to do conversions
// withouth involving otto code at all.
// nolint: errcheck, unparam
func (jail *Jail) Send(call otto.FunctionCall) otto.Value {
request, err := jail.vm.Call("JSON.stringify", nil, call.Argument(0))
value, err := cell.Run(code)
if err != nil {
throwJSException(err)
return newJailErrorResponse(err)
}
rpc := jail.nodeManager.RPCClient()
// TODO(divan): remove this check as soon as jail cells have
// proper cancellation mechanism implemented.
if rpc == nil {
throwJSException(fmt.Errorf("Error getting RPC client. Node stopped?"))
}
response := rpc.CallRaw(request.String())
return value.String()
}
// unmarshal response to pass to otto
var resp interface{}
err = json.Unmarshal([]byte(response), &resp)
// Call executes the `call` function within a cell with chatID.
// Returns a string being a valid JS code. In case of a successful result,
// it's {"result": any}. In case of an error: {"error": "some error"}.
//
// Call calls commands from `_status_catalog`.
// commandPath is an array of properties to retrieve a function.
// For instance:
// `["prop1", "prop2"]` is translated to `_status_catalog["prop1"]["prop2"]`.
func (j *Jail) Call(chatID, commandPath, args string) string {
cell, err := j.cell(chatID)
if err != nil {
throwJSException(fmt.Errorf("Error unmarshalling result: %s", err))
return newJailErrorResponse(err)
}
respValue, err := jail.vm.ToValue(resp)
value, err := cell.Call("call", nil, commandPath, args)
if err != nil {
throwJSException(fmt.Errorf("Error converting result to Otto's value: %s", err))
return newJailErrorResponse(err)
}
return respValue
return newJailResultResponse(value)
}
func newErrorResponse(msg string, id interface{}) map[string]interface{} {
// Bundle the error into a JSON RPC call response
return map[string]interface{}{
"jsonrpc": "2.0",
"id": id,
"error": map[string]interface{}{
"code": -32603, // Internal JSON-RPC Error, see http://www.jsonrpc.org/specification#error_object
"message": msg,
},
// RPCClient returns an rpc.Client.
func (j *Jail) RPCClient() *rpc.Client {
if j.rpcClientProvider == nil {
return nil
}
return j.rpcClientProvider.RPCClient()
}
func newErrorResponseOtto(vm *vm.VM, msg string, id interface{}) otto.Value {
// TODO(tiabc): Handle errors.
errResp, _ := json.Marshal(newErrorResponse(msg, id))
errRespVal, _ := vm.Run("(" + string(errResp) + ")")
return errRespVal
}
func newResultResponse(vm *otto.Otto, result interface{}) otto.Value {
resp, _ := vm.Object(`({"jsonrpc":"2.0"})`)
resp.Set("result", result) // nolint: errcheck
return resp.Value()
}
// throwJSException panics on an otto.Value. The Otto VM will recover from the
// Go panic and throw msg as a JavaScript error.
// nolint: unparam
func throwJSException(msg error) otto.Value {
val, err := otto.ToValue(msg.Error())
if err != nil {
log.Error(fmt.Sprintf("Failed to serialize JavaScript exception %v: %v", msg.Error(), err))
// sendRPCCall executes a raw JSON-RPC request.
func (j *Jail) sendRPCCall(request string) (interface{}, error) {
client := j.RPCClient()
if client == nil {
return nil, ErrNoRPCClient
}
panic(val)
rawResponse := client.CallRaw(request)
var response interface{}
if err := json.Unmarshal([]byte(rawResponse), &response); err != nil {
return nil, fmt.Errorf("failed to unmarshal response: %s", err)
}
return response, nil
}
// JSONError is wrapper around errors, that are sent upwards
type JSONError struct {
// newJailErrorResponse returns an error.
func newJailErrorResponse(err error) string {
response := struct {
Error string `json:"error"`
}
func makeError(error string) string {
str := JSONError{
Error: error,
}{
Error: err.Error(),
}
outBytes, _ := json.Marshal(&str)
return string(outBytes)
}
func makeResult(res string, err error) string {
var out string
rawResponse, err := json.Marshal(response)
if err != nil {
out = makeError(err.Error())
} else {
if "undefined" == res {
res = "null"
}
out = fmt.Sprintf(`{"result": %s}`, res)
return `{"error": "` + err.Error() + `"}`
}
return out
return string(rawResponse)
}
// newJailResultResponse returns a string that is a valid JavaScript code.
// Marshaling is not required as result.String() produces a string
// that is a valid JavaScript code.
func newJailResultResponse(result otto.Value) string {
return `{"result": ` + result.String() + `}`
}

156
geth/jail/jail_test.go Normal file
View File

@ -0,0 +1,156 @@
package jail
import (
"testing"
"github.com/robertkrimen/otto"
"github.com/status-im/status-go/geth/rpc"
"github.com/stretchr/testify/suite"
)
type testRPCClientProvider struct {
rpcClient *rpc.Client
}
func (p testRPCClientProvider) RPCClient() *rpc.Client {
return p.rpcClient
}
func TestJailTestSuite(t *testing.T) {
suite.Run(t, new(JailTestSuite))
}
type JailTestSuite struct {
suite.Suite
Jail *Jail
}
func (s *JailTestSuite) SetupTest() {
s.Jail = New(nil)
}
func (s *JailTestSuite) TestJailCreateCell() {
cell, err := s.Jail.CreateCell("cell1")
s.NoError(err)
s.NotNil(cell)
// creating another cell with the same id fails
_, err = s.Jail.CreateCell("cell1")
s.EqualError(err, "cell with id 'cell1' already exists")
// create more cells
_, err = s.Jail.CreateCell("cell2")
s.NoError(err)
_, err = s.Jail.CreateCell("cell3")
s.NoError(err)
s.Len(s.Jail.cells, 3)
}
func (s *JailTestSuite) TestJailGetCell() {
// cell1 does not exist
_, err := s.Jail.Cell("cell1")
s.EqualError(err, "cell 'cell1' not found")
// cell 1 exists
_, err = s.Jail.CreateCell("cell1")
s.NoError(err)
cell, err := s.Jail.Cell("cell1")
s.NoError(err)
s.NotNil(cell)
}
func (s *JailTestSuite) TestJailInitCell() {
// InitCell on an existing cell.
cell, err := s.Jail.createCell("cell1")
s.NoError(err)
err = s.Jail.initCell(cell)
s.NoError(err)
// web3 should be available
value, err := cell.Run("web3.fromAscii('ethereum')")
s.NoError(err)
s.Equal(`0x657468657265756d`, value.String())
}
func (s *JailTestSuite) TestJailStop() {
_, err := s.Jail.CreateCell("cell1")
s.NoError(err)
s.Len(s.Jail.cells, 1)
s.Jail.Stop()
s.Len(s.Jail.cells, 0)
}
func (s *JailTestSuite) TestJailCall() {
cell, err := s.Jail.CreateCell("cell1")
s.NoError(err)
propsc := make(chan string, 1)
argsc := make(chan string, 1)
err = cell.Set("call", func(call otto.FunctionCall) otto.Value {
propsc <- call.Argument(0).String()
argsc <- call.Argument(1).String()
return otto.UndefinedValue()
})
s.NoError(err)
result := s.Jail.Call("cell1", `["prop1", "prop2"]`, `arg1`)
s.Equal(`["prop1", "prop2"]`, <-propsc)
s.Equal(`arg1`, <-argsc)
s.Equal(`{"result": undefined}`, result)
}
func (s *JailTestSuite) TestMakeCatalogVariable() {
cell, err := s.Jail.createCell("cell1")
s.NoError(err)
// no `_status_catalog` variable
response := s.Jail.makeCatalogVariable(cell)
s.Equal(`{"error":"ReferenceError: '_status_catalog' is not defined"}`, response)
// with `_status_catalog` variable
_, err = cell.Run(`var _status_catalog = { test: true }`)
s.NoError(err)
response = s.Jail.makeCatalogVariable(cell)
s.Equal(`{"result": {"test":true}}`, response)
}
func (s *JailTestSuite) TestCreateAndInitCell() {
cell, err := s.Jail.createAndInitCell(
"cell1",
`var testCreateAndInitCell1 = true`,
`var testCreateAndInitCell2 = true`,
)
s.NoError(err)
s.NotNil(cell)
value, err := cell.Get("testCreateAndInitCell1")
s.NoError(err)
s.Equal(`true`, value.String())
value, err = cell.Get("testCreateAndInitCell2")
s.NoError(err)
s.Equal(`true`, value.String())
}
func (s *JailTestSuite) TestPublicCreateAndInitCell() {
response := s.Jail.CreateAndInitCell("cell1", `var _status_catalog = { test: true }`)
s.Equal(`{"result": {"test":true}}`, response)
}
func (s *JailTestSuite) TestExecute() {
// cell does not exist
response := s.Jail.Execute("cell1", "('some string')")
s.Equal(`{"error":"cell 'cell1' not found"}`, response)
_, err := s.Jail.createCell("cell1")
s.NoError(err)
// cell exists
response = s.Jail.Execute("cell1", `
var obj = { test: true };
JSON.stringify(obj);
`)
s.Equal(`{"test":true}`, response)
}

View File

@ -100,9 +100,14 @@ func (m *NodeManager) startNode(config *params.NodeConfig) (<-chan struct{}, err
m.config = config
// init RPC client for this node
m.rpcClient, err = rpc.NewClient(m.node, m.config.UpstreamConfig)
if err != nil {
log.Error("Init RPC client failed:", "error", err)
localRPCClient, errRPC := m.node.Attach()
if errRPC == nil {
m.rpcClient, errRPC = rpc.NewClient(localRPCClient, m.config.UpstreamConfig)
}
if errRPC != nil {
log.Error("Failed to create an RPC client", "error", errRPC)
m.Unlock()
signal.Send(signal.Envelope{
Type: signal.EventNodeCrashed,
@ -112,6 +117,7 @@ func (m *NodeManager) startNode(config *params.NodeConfig) (<-chan struct{}, err
})
return
}
m.Unlock()
// underlying node is started, every method can use it, we use it immediately

View File

@ -8,7 +8,6 @@ import (
"reflect"
"sync"
"github.com/ethereum/go-ethereum/node"
"github.com/status-im/status-go/geth/params"
gethrpc "github.com/ethereum/go-ethereum/rpc"
@ -38,21 +37,17 @@ type Client struct {
//
// Client is safe for concurrent use and will automatically
// reconnect to the server if connection is lost.
func NewClient(node *node.Node, upstream params.UpstreamRPCConfig) (*Client, error) {
c := &Client{
func NewClient(client *gethrpc.Client, upstream params.UpstreamRPCConfig) (*Client, error) {
c := Client{
local: client,
handlers: make(map[string]Handler),
}
var err error
c.local, err = node.Attach()
if err != nil {
return nil, fmt.Errorf("attach to local node: %s", err)
}
if upstream.Enabled {
c.upstreamEnabled = upstream.Enabled
c.upstreamURL = upstream.URL
c.upstream, err = gethrpc.Dial(c.upstreamURL)
if err != nil {
return nil, fmt.Errorf("dial upstream server: %s", err)
@ -61,7 +56,7 @@ func NewClient(node *node.Node, upstream params.UpstreamRPCConfig) (*Client, err
c.router = newRouter(c.upstreamEnabled)
return c, nil
return &c, nil
}
// Call performs a JSON-RPC call with the given arguments and unmarshals into

View File

@ -320,13 +320,20 @@ func DiscardTransactions(ids *C.char) *C.char {
//InitJail setup initial JavaScript
//export InitJail
func InitJail(js *C.char) {
statusAPI.JailBaseJS(C.GoString(js))
statusAPI.SetJailBaseJS(C.GoString(js))
}
//Parse creates a new jail cell context and executes provided JavaScript code
//export Parse
func Parse(chatID *C.char, js *C.char) *C.char {
res := statusAPI.JailParse(C.GoString(chatID), C.GoString(js))
//CreateAndInitCell creates a new jail cell context and executes provided JavaScript code.
//export CreateAndInitCell
func CreateAndInitCell(chatID *C.char, js *C.char) *C.char {
res := statusAPI.CreateAndInitCell(C.GoString(chatID), C.GoString(js))
return C.CString(res)
}
//ExecuteJS allows to run arbitrary JS code within a cell.
//export ExecuteJS
func ExecuteJS(chatID *C.char, code *C.char) *C.char {
res := statusAPI.JailExecute(C.GoString(chatID), C.GoString(code))
return C.CString(res)
}

View File

@ -11,6 +11,7 @@ import (
"path/filepath"
"reflect"
"strconv"
"strings"
"testing"
"time"
@ -114,6 +115,10 @@ func testExportedAPI(t *testing.T, done chan struct{}) {
"test jailed calls",
testJailFunctionCall,
},
{
"test ExecuteJS",
testExecuteJS,
},
}
for _, test := range tests {
@ -1231,12 +1236,12 @@ func testJailInitInvalid(t *testing.T) bool {
// Act.
InitJail(C.CString(initInvalidCode))
response := C.GoString(Parse(C.CString("CHAT_ID_INIT_TEST"), C.CString(``)))
response := C.GoString(CreateAndInitCell(C.CString("CHAT_ID_INIT_INVALID_TEST"), C.CString(``)))
// Assert.
expectedResponse := `{"error":"(anonymous): Line 4:3 Unexpected end of input (and 3 more errors)"}`
if expectedResponse != response {
t.Errorf("unexpected response, expected: %v, got: %v", expectedResponse, response)
expectedSubstr := `"error":"(anonymous): Line 4:3 Unexpected identifier`
if !strings.Contains(response, expectedSubstr) {
t.Errorf("unexpected response, didn't find '%s' in '%s'", expectedSubstr, response)
return false
}
return true
@ -1256,11 +1261,10 @@ func testJailParseInvalid(t *testing.T) bool {
var extraFunc = function (x) {
return x * x;
`
response := C.GoString(Parse(C.CString("CHAT_ID_INIT_TEST"), C.CString(extraInvalidCode)))
response := C.GoString(CreateAndInitCell(C.CString("CHAT_ID_PARSE_INVALID_TEST"), C.CString(extraInvalidCode)))
// Assert.
// expectedResponse := `{"error":"(anonymous): Line 16331:50 Unexpected end of input (and 1 more errors)"}`
expectedResponse := `{"error":"(anonymous): Line 16354:50 Unexpected end of input (and 1 more errors)"}`
expectedResponse := `{"error":"(anonymous): Line 4:2 Unexpected end of input (and 1 more errors)"}`
if expectedResponse != response {
t.Errorf("unexpected response, expected: %v, got: %v", expectedResponse, response)
return false
@ -1281,13 +1285,13 @@ func testJailInit(t *testing.T) bool {
return x * x;
};
`
rawResponse := Parse(C.CString("CHAT_ID_INIT_TEST"), C.CString(extraCode))
rawResponse := CreateAndInitCell(C.CString("CHAT_ID_INIT_TEST"), C.CString(extraCode))
parsedResponse := C.GoString(rawResponse)
expectedResponse := `{"result": {"foo":"bar"}}`
if !reflect.DeepEqual(expectedResponse, parsedResponse) {
t.Error("expected output not returned from jail.Parse()")
t.Error("expected output not returned from jail.CreateAndInitCell()")
return false
}
@ -1304,12 +1308,12 @@ func testJailFunctionCall(t *testing.T) bool {
_status_catalog.commands["testCommand"] = function (params) {
return params.val * params.val;
};`
Parse(C.CString("CHAT_ID_CALL_TEST"), C.CString(statusJS))
CreateAndInitCell(C.CString("CHAT_ID_CALL_TEST"), C.CString(statusJS))
// call with wrong chat id
rawResponse := Call(C.CString("CHAT_IDNON_EXISTENT"), C.CString(""), C.CString(""))
parsedResponse := C.GoString(rawResponse)
expectedError := `{"error":"cell[CHAT_IDNON_EXISTENT] doesn't exist"}`
expectedError := `{"error":"cell 'CHAT_IDNON_EXISTENT' not found"}`
if parsedResponse != expectedError {
t.Errorf("expected error is not returned: expected %s, got %s", expectedError, parsedResponse)
return false
@ -1329,6 +1333,30 @@ func testJailFunctionCall(t *testing.T) bool {
return true
}
func testExecuteJS(t *testing.T) bool {
InitJail(C.CString(""))
// cell does not exist
response := C.GoString(ExecuteJS(C.CString("CHAT_ID_EXECUTE_TEST"), C.CString("('some string')")))
expectedResponse := `{"error":"cell 'CHAT_ID_EXECUTE_TEST' not found"}`
if response != expectedResponse {
t.Errorf("expected '%s' but got '%s'", expectedResponse, response)
return false
}
CreateAndInitCell(C.CString("CHAT_ID_EXECUTE_TEST"), C.CString(`var obj = { status: true }`))
// cell does not exist
response = C.GoString(ExecuteJS(C.CString("CHAT_ID_EXECUTE_TEST"), C.CString(`JSON.stringify(obj)`)))
expectedResponse = `{"status":true}`
if response != expectedResponse {
t.Errorf("expected '%s' but got '%s'", expectedResponse, response)
return false
}
return true
}
func startTestNode(t *testing.T) <-chan struct{} {
syncRequired := false
if _, err := os.Stat(TestDataDir); os.IsNotExist(err) {