You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by cc...@apache.org on 2016/10/13 21:41:46 UTC

incubator-mynewt-newt git commit: imgmgmt - Replace obsolete cmds with image state.

Repository: incubator-mynewt-newt
Updated Branches:
  refs/heads/develop c49488c77 -> 868eda0ff


imgmgmt - Replace obsolete cmds with image state.


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/commit/868eda0f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/tree/868eda0f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/diff/868eda0f

Branch: refs/heads/develop
Commit: 868eda0fff74de73a7fa524bd5085fdca09bdfdf
Parents: c49488c
Author: Christopher Collins <cc...@apache.org>
Authored: Thu Oct 13 14:39:42 2016 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Thu Oct 13 14:41:38 2016 -0700

----------------------------------------------------------------------
 newtmgr/cli/commands.go        |   1 -
 newtmgr/cli/image.go           | 267 +++++++++++++++---------------------
 newtmgr/cli/split.go           | 103 --------------
 newtmgr/protocol/imageboot2.go |  94 -------------
 newtmgr/protocol/imagedefs.go  |  77 +++++++++++
 newtmgr/protocol/imagelist.go  |  32 -----
 newtmgr/protocol/imagelist2.go |  71 ----------
 newtmgr/protocol/imagesplit.go | 183 ------------------------
 newtmgr/protocol/imagestate.go |   8 --
 9 files changed, 189 insertions(+), 647 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/cli/commands.go
----------------------------------------------------------------------
diff --git a/newtmgr/cli/commands.go b/newtmgr/cli/commands.go
index 58f8a9f..bbcb4a1 100644
--- a/newtmgr/cli/commands.go
+++ b/newtmgr/cli/commands.go
@@ -59,7 +59,6 @@ func Commands() *cobra.Command {
 	nmCmd.AddCommand(connProfileCmd())
 	nmCmd.AddCommand(echoCmd())
 	nmCmd.AddCommand(imageCmd())
-	nmCmd.AddCommand(splitCmd())
 	nmCmd.AddCommand(statsCmd())
 	nmCmd.AddCommand(taskStatsCmd())
 	nmCmd.AddCommand(mempoolStatsCmd())

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/cli/image.go
----------------------------------------------------------------------
diff --git a/newtmgr/cli/image.go b/newtmgr/cli/image.go
index 48090f9..7e02407 100644
--- a/newtmgr/cli/image.go
+++ b/newtmgr/cli/image.go
@@ -44,147 +44,153 @@ var (
 	coreNumBytes uint32
 )
 
-func imageListCmd(cmd *cobra.Command, args []string) {
+func imageFlagsStr(image protocol.ImageStateEntry) string {
+	strs := []string{}
+
+	if image.Active {
+		strs = append(strs, "active")
+	}
+	if image.Confirmed {
+		strs = append(strs, "confirmed")
+	}
+	if image.Pending {
+		strs = append(strs, "pending")
+	}
+
+	return strings.Join(strs, " ")
+}
+
+func imageStatePrintRsp(rsp *protocol.ImageStateRsp) error {
+	if rsp.ReturnCode != 0 {
+		return util.FmtNewtError("rc=%d\n", rsp.ReturnCode)
+	}
+	fmt.Println("Images:")
+	for _, img := range rsp.Images {
+		fmt.Printf(" slot=%d\n", img.Slot)
+		fmt.Printf("    version: %s\n", img.Version)
+		fmt.Printf("    bootable: %v\n", img.Bootable)
+		fmt.Printf("    flags: %s\n", imageFlagsStr(img))
+		if len(img.Hash) == 0 {
+			fmt.Printf("    hash: Unavailable\n")
+		} else {
+			fmt.Printf("    hash: %x\n", img.Hash)
+		}
+	}
+
+	fmt.Printf("Split status: %s\n", rsp.SplitStatus.String())
+	return nil
+}
+
+func imageStateListCmd(cmd *cobra.Command, args []string) {
 	runner, err := getTargetCmdRunner()
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 	defer runner.Conn.Close()
 
-	imageList, err := protocol.NewImageList2()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
+	var nmr *protocol.NmgrReq
 
-	nmr, err := imageList.EncodeWriteRequest()
+	req := protocol.ImageStateReadReq{}
+	nmr, err = req.Encode()
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 
 	if err := runner.WriteReq(nmr); err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 
-	rsp, err := runner.ReadResp()
+	rawRsp, err := runner.ReadResp()
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 
-	iRsp, err := protocol.DecodeImageListResponse2(rsp.Data)
+	rsp, err := protocol.DecodeImageStateResponse(rawRsp.Data)
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
-	fmt.Println("Images:")
-	for _, img := range iRsp.Images {
-
-		fmt.Printf(" slot=%d\n", img.Slot)
-		fmt.Printf("    version=%s\n", img.Version)
-		fmt.Printf("    bootable=%v\n", img.Bootable)
-		if len(img.Hash) == 0 {
-			fmt.Printf("    hash=Unavailable\n")
-		} else {
-			fmt.Printf("    hash=%x\n", img.Hash)
-		}
+	if err := imageStatePrintRsp(rsp); err != nil {
+		nmUsage(nil, err)
 	}
 }
 
-func imageFlagsStr(image protocol.ImageStateEntry) string {
-	strs := []string{}
-
-	if image.Active {
-		strs = append(strs, "active")
+func imageStateTestCmd(cmd *cobra.Command, args []string) {
+	if len(args) < 1 {
+		nmUsage(cmd, nil)
 	}
-	if image.Confirmed {
-		strs = append(strs, "confirmed")
+
+	hex_bytes, _ := hex.DecodeString(args[0])
+
+	req := protocol.ImageStateWriteReq{
+		Hash:    hex_bytes,
+		Confirm: false,
 	}
-	if image.Pending {
-		strs = append(strs, "pending")
+	nmr, err := req.Encode()
+	if err != nil {
+		nmUsage(nil, err)
 	}
 
-	return strings.Join(strs, " ")
-}
-
-func imageStateCmd(cmd *cobra.Command, args []string) {
 	runner, err := getTargetCmdRunner()
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 	defer runner.Conn.Close()
 
-	var nmr *protocol.NmgrReq
+	if err := runner.WriteReq(nmr); err != nil {
+		nmUsage(nil, err)
+	}
 
-	if len(args) == 0 {
-		nmUsage(cmd, nil)
+	rawRsp, err := runner.ReadResp()
+	if err != nil {
+		nmUsage(nil, err)
 	}
 
-	if args[0] == "show" {
-		req := protocol.ImageStateReadReq{}
-		nmr, err = req.Encode()
-		if err != nil {
-			nmUsage(cmd, err)
-		}
-	} else if args[0] == "test" {
-		if len(args) < 2 {
-			nmUsage(cmd, nil)
-		}
+	rsp, err := protocol.DecodeImageStateResponse(rawRsp.Data)
+	if err != nil {
+		nmUsage(nil, err)
+	}
+	if err := imageStatePrintRsp(rsp); err != nil {
+		nmUsage(nil, err)
+	}
+}
 
-		hex_bytes, _ := hex.DecodeString(args[1])
+func imageStateConfirmCmd(cmd *cobra.Command, args []string) {
+	req := protocol.ImageStateWriteReq{
+		Hash:    nil,
+		Confirm: true,
+	}
+	nmr, err := req.Encode()
+	if err != nil {
+		nmUsage(cmd, err)
+	}
 
-		req := protocol.ImageStateWriteReq{
-			Hash:    hex_bytes,
-			Confirm: false,
-		}
-		nmr, err = req.Encode()
-		if err != nil {
-			nmUsage(cmd, err)
-		}
-	} else if args[0] == "confirm" {
-		req := protocol.ImageStateWriteReq{
-			Hash:    make([]byte, 0),
-			Confirm: true,
-		}
-		nmr, err = req.Encode()
-		if err != nil {
-			nmUsage(cmd, err)
-		}
-	} else {
-		nmUsage(cmd, nil)
+	runner, err := getTargetCmdRunner()
+	if err != nil {
+		nmUsage(nil, err)
 	}
+	defer runner.Conn.Close()
 
 	if err := runner.WriteReq(nmr); err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 
-	rsp, err := runner.ReadResp()
+	rawRsp, err := runner.ReadResp()
 	if err != nil {
-		nmUsage(cmd, err)
+		nmUsage(nil, err)
 	}
 
-	iRsp, err := protocol.DecodeImageStateResponse(rsp.Data)
+	rsp, err := protocol.DecodeImageStateResponse(rawRsp.Data)
 	if err != nil {
-		nmUsage(cmd, err)
-	}
-	if iRsp.ReturnCode != 0 {
-		fmt.Printf("Error executing state command: rc=%d\n", iRsp.ReturnCode)
-		return
+		nmUsage(nil, err)
 	}
-	fmt.Println("Images:")
-	for _, img := range iRsp.Images {
-		fmt.Printf(" slot=%d\n", img.Slot)
-		fmt.Printf("    version: %s\n", img.Version)
-		fmt.Printf("    bootable: %v\n", img.Bootable)
-		fmt.Printf("    flags: %s\n", imageFlagsStr(img))
-		if len(img.Hash) == 0 {
-			fmt.Printf("    hash: Unavailable\n")
-		} else {
-			fmt.Printf("    hash: %x\n", img.Hash)
-		}
+	if err := imageStatePrintRsp(rsp); err != nil {
+		nmUsage(nil, err)
 	}
-
-	fmt.Printf("Split status: %s\n", iRsp.SplitStatus.String())
 }
 
-func echoOnNmUsage(runner *protocol.CmdRunner, cmderr error, cmd *cobra.Command) {
+func echoOnNmUsage(
+	runner *protocol.CmdRunner, cmderr error, cmd *cobra.Command) {
+
 	echoCtrl(runner, "1")
 	nmUsage(cmd, cmderr)
 }
@@ -313,46 +319,6 @@ func imageUploadCmd(cmd *cobra.Command, args []string) {
 	fmt.Println("Done")
 }
 
-func imageBootCmd(cmd *cobra.Command, args []string) {
-	runner, err := getTargetCmdRunner()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-	defer runner.Conn.Close()
-
-	imageBoot, err := protocol.NewImageBoot2()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-
-	if len(args) >= 1 {
-		imageBoot.BootTarget, _ = hex.DecodeString(args[0])
-	}
-	nmr, err := imageBoot.EncodeWriteRequest()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-
-	if err := runner.WriteReq(nmr); err != nil {
-		nmUsage(cmd, err)
-	}
-
-	rsp, err := runner.ReadResp()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-
-	iRsp, err := protocol.DecodeImageBoot2Response(rsp.Data)
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-	if len(args) == 0 {
-		fmt.Printf("   Test image: %x\n", iRsp.Test)
-		fmt.Printf("   Main image: %x\n", iRsp.Main)
-		fmt.Printf("   Active img: %x\n", iRsp.Active)
-	}
-}
-
 func fileUploadCmd(cmd *cobra.Command, args []string) {
 	if len(args) < 2 {
 		nmUsage(cmd, util.NewNewtError(
@@ -665,21 +631,23 @@ func imageCmd() *cobra.Command {
 	listCmd := &cobra.Command{
 		Use:   "list",
 		Short: "Show target images",
-		Run:   imageListCmd,
+		Run:   imageStateListCmd,
 	}
 	imageCmd.AddCommand(listCmd)
 
-	stateEx := "  newtmgr -c olimex image state show\n"
-	stateEx += "  newtmgr -c olimex image state test <hash>\n"
-	stateEx += "  newtmgr -c olimex image state confirm"
+	testCmd := &cobra.Command{
+		Use:   "test <hex-image-hash>",
+		Short: "Test an image on next reboot",
+		Run:   imageStateTestCmd,
+	}
+	imageCmd.AddCommand(testCmd)
 
-	stateCmd := &cobra.Command{
-		Use:     "state",
-		Short:   "Show target images",
-		Example: stateEx,
-		Run:     imageStateCmd,
+	confirmCmd := &cobra.Command{
+		Use:   "confirm",
+		Short: "Confirm current image setup",
+		Run:   imageStateConfirmCmd,
 	}
-	imageCmd.AddCommand(stateCmd)
+	imageCmd.AddCommand(confirmCmd)
 
 	uploadEx := "  newtmgr -c olimex image upload <image_file\n"
 	uploadEx += "  newtmgr -c olimex image upload bin/slinky_zero/apps/slinky.img\n"
@@ -692,17 +660,6 @@ func imageCmd() *cobra.Command {
 	}
 	imageCmd.AddCommand(uploadCmd)
 
-	bootEx := "  newtmgr -c olimex image boot [<image hash>]\n"
-	bootEx += "  newtmgr -c olimex image boot\n"
-
-	bootCmd := &cobra.Command{
-		Use:     "boot",
-		Short:   "Which image to boot",
-		Example: bootEx,
-		Run:     imageBootCmd,
-	}
-	imageCmd.AddCommand(bootCmd)
-
 	fileUploadEx := "  newtmgr -c olimex image fileupload <filename> <tgt_file>\n"
 	fileUploadEx += "  newtmgr -c olimex image fileupload sample.lua /sample.lua\n"
 

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/cli/split.go
----------------------------------------------------------------------
diff --git a/newtmgr/cli/split.go b/newtmgr/cli/split.go
deleted file mode 100644
index bd16a52..0000000
--- a/newtmgr/cli/split.go
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package cli
-
-import (
-	"fmt"
-
-	"github.com/spf13/cobra"
-	"mynewt.apache.org/newt/newtmgr/protocol"
-	"mynewt.apache.org/newt/util"
-)
-
-func splitStatusCmd(cmd *cobra.Command, args []string) {
-	runner, err := getTargetCmdRunner()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-	defer runner.Conn.Close()
-
-	split, err := protocol.NewSplit()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-	var nmr *protocol.NmgrReq
-	if len(args) == 0 {
-		nmr, err = split.EncoderReadRequest()
-	} else if len(args) == 1 {
-		b, err := protocol.ParseSplitMode(args[0])
-
-		if err != nil {
-			nmUsage(cmd, util.NewNewtError("Invalid Boolean Argument"))
-		}
-		split.Split = b
-		nmr, err = split.EncoderWriteRequest()
-	} else {
-		nmUsage(cmd, nil)
-		return
-	}
-
-	if err := runner.WriteReq(nmr); err != nil {
-		nmUsage(cmd, err)
-	}
-
-	rsp, err := runner.ReadResp()
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-
-	srsp, err := protocol.DecodeSplitReadResponse(rsp.Data)
-	if err != nil {
-		nmUsage(cmd, err)
-	}
-
-	if len(args) == 0 {
-		fmt.Printf("Split value is %s\n", srsp.Split)
-		fmt.Printf("Split status is %s\n", srsp.Status)
-
-	}
-	if srsp.ReturnCode != 0 {
-		fmt.Printf("Error executing split command: rc=%d\n", srsp.ReturnCode)
-	}
-}
-
-func splitCmd() *cobra.Command {
-	splitImgCmd := &cobra.Command{
-		Use:   "split",
-		Short: "Manage split images on remote instance",
-		Run: func(cmd *cobra.Command, args []string) {
-			cmd.HelpFunc()(cmd, args)
-		},
-	}
-
-	splitEx := "  newtmgr -c olimex image split 1\n"
-	splitEx += "  newtmgr -c olimex image split 0\n"
-	splitEx += "  newtmgr -c olimex image split\n"
-
-	splitStatusCmd := &cobra.Command{
-		Use:     "status",
-		Short:   "Erase core on target",
-		Example: splitEx,
-		Run:     splitStatusCmd,
-	}
-	splitImgCmd.AddCommand(splitStatusCmd)
-
-	return splitImgCmd
-}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imageboot2.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imageboot2.go b/newtmgr/protocol/imageboot2.go
deleted file mode 100644
index 3ded958..0000000
--- a/newtmgr/protocol/imageboot2.go
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package protocol
-
-import (
-	"fmt"
-
-	"github.com/ugorji/go/codec"
-	"mynewt.apache.org/newt/util"
-)
-
-type ImageBoot2 struct {
-	BootTarget []byte
-	Test       []byte `codec:"test"`
-	Main       []byte `codec:"main"`
-	Active     []byte `codec:"active"`
-	ReturnCode int    `codec:"rc"`
-}
-
-func NewImageBoot2() (*ImageBoot2, error) {
-	s := &ImageBoot2{}
-	s.BootTarget = make([]byte, 0)
-	s.Test = make([]byte, 0)
-	s.Main = make([]byte, 0)
-	s.Active = make([]byte, 0)
-	return s, nil
-}
-
-func (i *ImageBoot2) EncodeWriteRequest() (*NmgrReq, error) {
-	nmr, err := NewNmgrReq()
-	if err != nil {
-		return nil, err
-	}
-
-	nmr.Op = NMGR_OP_READ
-	nmr.Flags = 0
-	nmr.Group = NMGR_GROUP_ID_IMAGE
-	nmr.Id = IMGMGR_NMGR_OP_BOOT2
-	nmr.Len = 0
-
-	if len(i.BootTarget) != 0 {
-		type BootReq struct {
-			Test []byte `codec:"test"`
-		}
-
-		bReq := &BootReq{
-			Test: i.BootTarget,
-		}
-		data := make([]byte, 0)
-		enc := codec.NewEncoderBytes(&data, new(codec.CborHandle))
-		enc.Encode(bReq)
-		nmr.Data = data
-		nmr.Len = uint16(len(data))
-		nmr.Op = NMGR_OP_WRITE
-	}
-	return nmr, nil
-}
-
-func DecodeImageBoot2Response(data []byte) (*ImageBoot2, error) {
-	i := &ImageBoot2{}
-
-	if len(data) == 0 {
-		return i, nil
-	}
-	dec := codec.NewDecoderBytes(data, new(codec.CborHandle))
-	err := dec.Decode(&i)
-	if err != nil {
-		return nil, util.NewNewtError(fmt.Sprintf("Invalid incoming cbor: %s",
-			err.Error()))
-	}
-	if i.ReturnCode != 0 {
-		return nil, util.NewNewtError(fmt.Sprintf("Target error: %d",
-			i.ReturnCode))
-	}
-
-	return i, nil
-}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imagedefs.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imagedefs.go b/newtmgr/protocol/imagedefs.go
new file mode 100644
index 0000000..46f43c2
--- /dev/null
+++ b/newtmgr/protocol/imagedefs.go
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package protocol
+
+import (
+	"encoding/base64"
+	"encoding/hex"
+	"fmt"
+
+	"mynewt.apache.org/newt/util"
+)
+
+const (
+	IMGMGR_NMGR_OP_STATE    = 0
+	IMGMGR_NMGR_OP_UPLOAD   = 1
+	IMGMGR_NMGR_OP_FILE     = 2
+	IMGMGR_NMGR_OP_CORELIST = 3
+	IMGMGR_NMGR_OP_CORELOAD = 4
+)
+
+type SplitStatus int
+
+const (
+	NOT_APPLICABLE SplitStatus = iota
+	NOT_MATCHING
+	MATCHING
+)
+
+/* returns the enum as a string */
+func (sm SplitStatus) String() string {
+	names := map[SplitStatus]string{
+		NOT_APPLICABLE: "N/A",
+		NOT_MATCHING:   "non-matching",
+		MATCHING:       "matching",
+	}
+
+	str := names[sm]
+	if str == "" {
+		return "Unknown!"
+	}
+	return str
+}
+
+func HashDecode(src string) (string, error) {
+	imgHex, err := base64.StdEncoding.DecodeString(src)
+	if err != nil {
+		return "", util.NewNewtError(fmt.Sprintf("Hash decode error: %s",
+			err.Error()))
+	}
+	return hex.EncodeToString(imgHex), nil
+}
+
+func HashEncode(src string) (string, error) {
+	imgHex, err := hex.DecodeString(src)
+	if err != nil {
+		return "", util.NewNewtError(fmt.Sprintf("Hash encode error: %s",
+			err.Error()))
+	}
+	return base64.StdEncoding.EncodeToString(imgHex), nil
+}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imagelist.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imagelist.go b/newtmgr/protocol/imagelist.go
deleted file mode 100644
index 1c31bf8..0000000
--- a/newtmgr/protocol/imagelist.go
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package protocol
-
-const (
-	IMGMGR_NMGR_OP_LIST     = 0
-	IMGMGR_NMGR_OP_UPLOAD   = 1
-	IMGMGR_NMGR_OP_BOOT     = 2
-	IMGMGR_NMGR_OP_FILE     = 3
-	IMGMGR_NMGR_OP_LIST2    = 4
-	IMGMGR_NMGR_OP_BOOT2    = 5
-	IMGMGR_NMGR_OP_CORELIST = 6
-	IMGMGR_NMGR_OP_CORELOAD = 7
-	IMGMGR_NMGR_OP_STATE    = 8
-)

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imagelist2.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imagelist2.go b/newtmgr/protocol/imagelist2.go
deleted file mode 100644
index 04bb0b4..0000000
--- a/newtmgr/protocol/imagelist2.go
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package protocol
-
-import (
-	"fmt"
-
-	"github.com/ugorji/go/codec"
-	"mynewt.apache.org/newt/util"
-)
-
-type Image2 struct {
-	Slot     int    `codec:"slot"`
-	Version  string `codec:"version"`
-	Hash     []byte `codec:"hash"`
-	Bootable bool   `codec:"bootable"`
-}
-
-type ImageList2 struct {
-	Images []Image2 `codec:"images"`
-}
-
-func NewImageList2() (*ImageList2, error) {
-	s := &ImageList2{}
-	return s, nil
-}
-
-func (i *ImageList2) EncodeWriteRequest() (*NmgrReq, error) {
-	nmr, err := NewNmgrReq()
-	if err != nil {
-		return nil, err
-	}
-
-	nmr.Op = NMGR_OP_READ
-	nmr.Flags = 0
-	nmr.Group = NMGR_GROUP_ID_IMAGE
-	nmr.Id = IMGMGR_NMGR_OP_LIST2
-	nmr.Len = 0
-
-	return nmr, nil
-}
-
-func DecodeImageListResponse2(data []byte) (*ImageList2, error) {
-
-	list2 := &ImageList2{}
-
-	dec := codec.NewDecoderBytes(data, new(codec.CborHandle))
-	err := dec.Decode(&list2)
-	if err != nil {
-		return nil, util.NewNewtError(fmt.Sprintf("Invalid incoming cbor: %s",
-			err.Error()))
-	}
-	return list2, nil
-}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imagesplit.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imagesplit.go b/newtmgr/protocol/imagesplit.go
deleted file mode 100644
index 7c91cce..0000000
--- a/newtmgr/protocol/imagesplit.go
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package protocol
-
-import (
-	"fmt"
-	"strings"
-
-	"github.com/ugorji/go/codec"
-	"mynewt.apache.org/newt/util"
-)
-
-const (
-	SPLIT_NMGR_OP_SPLIT = 0
-)
-
-type SplitMode int
-
-const (
-	NONE SplitMode = iota
-	TEST
-	RUN
-)
-
-var splitMode = [...]string{NONE: "none", TEST: "test", RUN: "run"}
-
-/* is the enum valid */
-func (sm SplitMode) Valid() bool {
-	for val, _ := range splitMode {
-		if int(sm) == val {
-			return true
-		}
-	}
-	return false
-}
-
-/* returns the enum as a string */
-func (sm SplitMode) String() string {
-	if sm > RUN || sm < 0 {
-		return "Invalid!"
-	}
-	return splitMode[sm]
-}
-
-type SplitStatus int
-
-const (
-	NOT_APPLICABLE SplitStatus = iota
-	NOT_MATCHING
-	MATCHING
-)
-
-/* parses the enum from a string */
-func ParseSplitMode(str string) (SplitMode, error) {
-	for val, key := range splitMode {
-		if strings.EqualFold(key, str) {
-			return SplitMode(val), nil
-		}
-	}
-	return NONE, util.NewNewtError("Invalid value for Split Mode %v" + str)
-}
-
-var splitStatus = [...]string{
-	NOT_APPLICABLE: "N/A",
-	NOT_MATCHING:   "Non-matching",
-	MATCHING:       "matching",
-}
-
-/* is the enum valid */
-func (sm SplitStatus) Valid() bool {
-	for val, _ := range splitStatus {
-		if int(sm) == val {
-			return true
-		}
-	}
-	return false
-}
-
-/* returns the enum as a string */
-func (sm SplitStatus) String() string {
-	if sm > MATCHING || sm < 0 {
-		return "Unknown!"
-	}
-	return splitStatus[sm]
-}
-
-/* parses the enum from a string */
-func ParseSplitStatus(str string) (SplitStatus, error) {
-	for val, key := range splitStatus {
-		if strings.EqualFold(key, str) {
-			return SplitStatus(val), nil
-		}
-	}
-	return NOT_APPLICABLE, util.NewNewtError("Invalid value for Split Status %v" + str)
-}
-
-type Split struct {
-	Split      SplitMode   `codec:"splitMode"`
-	Status     SplitStatus `codec:"splitStatus"`
-	ReturnCode int         `codec:"rc"`
-}
-
-func NewSplit() (*Split, error) {
-	s := &Split{}
-	return s, nil
-}
-
-func (s *Split) EncoderReadRequest() (*NmgrReq, error) {
-
-	nmr, err := NewNmgrReq()
-	if err != nil {
-		return nil, err
-	}
-
-	nmr.Op = NMGR_OP_READ
-	nmr.Flags = 0
-	nmr.Group = NMGR_GROUP_ID_SPLIT
-	nmr.Id = SPLIT_NMGR_OP_SPLIT
-	nmr.Len = 0
-
-	return nmr, nil
-}
-
-func (s *Split) EncoderWriteRequest() (*NmgrReq, error) {
-
-	data := make([]byte, 0)
-	enc := codec.NewEncoderBytes(&data, new(codec.CborHandle))
-	err := enc.Encode(s)
-	if err != nil {
-		return nil, err
-	}
-
-	nmr, err := NewNmgrReq()
-	if err != nil {
-		return nil, err
-	}
-
-	nmr.Op = NMGR_OP_WRITE
-	nmr.Flags = 0
-	nmr.Group = NMGR_GROUP_ID_SPLIT
-	nmr.Id = SPLIT_NMGR_OP_SPLIT
-	nmr.Len = uint16(len(data))
-	nmr.Data = data
-
-	return nmr, nil
-}
-
-func DecodeSplitReadResponse(data []byte) (*Split, error) {
-	i := &Split{}
-
-	if len(data) == 0 {
-		return i, nil
-	}
-
-	dec := codec.NewDecoderBytes(data, new(codec.CborHandle))
-	err := dec.Decode(&i)
-	if err != nil {
-		return nil, util.NewNewtError(fmt.Sprintf("Invalid incoming cbor: %s",
-			err.Error()))
-	}
-	if i.ReturnCode != 0 {
-		return nil, util.NewNewtError(fmt.Sprintf("Target error: %d",
-			i.ReturnCode))
-	}
-	return i, nil
-}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/868eda0f/newtmgr/protocol/imagestate.go
----------------------------------------------------------------------
diff --git a/newtmgr/protocol/imagestate.go b/newtmgr/protocol/imagestate.go
index dc2cff9..e90109e 100644
--- a/newtmgr/protocol/imagestate.go
+++ b/newtmgr/protocol/imagestate.go
@@ -26,14 +26,6 @@ import (
 	"mynewt.apache.org/newt/util"
 )
 
-//type SplitStatus int
-
-//const (
-//NOT_APPLICABLE SplitStatus = iota
-//NOT_MATCHING
-//MATCHING
-//)
-
 type ImageStateEntry struct {
 	Slot      int    `codec:"slot"`
 	Version   string `codec:"version"`