You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/11/08 12:42:41 UTC

[GitHub] [tvm] mbaret opened a new pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

mbaret opened a new pull request #9469:
URL: https://github.com/apache/tvm/pull/9469


   RFC: apache/tvm-rfcs#37
   Issue: #9429
   
   NOTE: This PR builds on top of https://github.com/apache/tvm/pull/9458 and therefore includes those changes. This PR will remain as 'draft' until the dependency is merged.
   
   A CascaderGraph augments a TE graph with additional information needed by the cascading algorithms. This includes defining a strict ordering on the operators as well as including all the Propagators needed to do the affine analysis of cascades.
   
   The CascaderGraph consists of two object types, Parts and Tensors. A Part is an augmented operator which includes the Propagators and a Tensor is similar to a TE tensor but stores additional information like compression ratio.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r773771558



##########
File path: src/contrib/ethosu/cascader/common.h
##########
@@ -103,6 +119,18 @@ inline std::size_t hash_vector(const std::vector<T>& vec) {
   return seed;
 }
 
+template <class T>
+inline T mul_reduce(const std::vector<T>& vec) {
+  if (vec.size() == 0) {
+    return 0;
+  }
+  T v = vec[0];
+  for (unsigned int i = 1; i < vec.size(); i++) {
+    v *= vec[i];
+  }
+  return v;
+}

Review comment:
       I've kept the function as I think it's a useful shorthand, but have changed the implementation to your suggestion.

##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.
+ * \note The performance information for a Part is composed of 3 factors: the compute cycles,
+ * the number of bytes read from each input tensor and the number of bytes written to the output
+ * tensor. Bytes read/written is reported in favour of read/write bandwidth cycles so the
+ * calculation of the performance information can be re-used with different memory homing.
+ */
+class PerformanceInfo : public ObjectRef {
+ public:
+  PerformanceInfo(size_t compute_cycles, std::vector<size_t> read_bytes, size_t write_bytes) {
+    auto n = make_object<PerformanceInfoNode>();
+    n->compute_cycles = compute_cycles;
+    n->read_bytes = std::move(read_bytes);
+    n->write_bytes = write_bytes;
+    data_ = std::move(n);
+  }
+
+  TVM_DEFINE_OBJECT_REF_METHODS(PerformanceInfo, ObjectRef, PerformanceInfoNode);
+};
+
+/*! \brief Node to represent a Tensor */
+class TensorNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \return The shape of the tensor */

Review comment:
       Making the members public isn't enough on its own to reflect them into Python, PerformanceInfo still defines VisitAttrs. Tensor having MUTABLE_OBJECT_REF is so that AddProducer/Consumer work as these are non-const methods.

##########
File path: cmake/modules/contrib/EthosU.cmake
##########
@@ -18,7 +18,8 @@
 if(USE_ETHOSU)
   file(GLOB COMPILER_ETHOSU_SRCS
        CONFIGURE_DEPENDS src/relay/backend/contrib/ethosu/*
-       CONFIGURE_DEPENDS src/contrib/ethosu/cascader/*)
+       CONFIGURE_DEPENDS src/contrib/ethosu/cascader/*

Review comment:
       Understood, good catch. I've added these files to the 'else' condition here.

##########
File path: python/tvm/contrib/ethosu/cascader/graph.py
##########
@@ -0,0 +1,170 @@
+# 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.
+"""Graph objects to define compute graphs for the NPU cascader."""
+from typing import List
+from collections import namedtuple
+import tvm._ffi
+
+from tvm.runtime import Object
+
+from .stripe_config import StripeConfig
+from . import _ffi_api
+
+
+TESubgraph = namedtuple("TESubgraph", ["input_tensors", "output_tensor"])
+
+
+@tvm._ffi.register_object("contrib.ethosu.cascader.PerformanceInfo")
+class PerformanceInfo(Object):

Review comment:
       ack




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772706923



##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.
+ * \note The performance information for a Part is composed of 3 factors: the compute cycles,
+ * the number of bytes read from each input tensor and the number of bytes written to the output
+ * tensor. Bytes read/written is reported in favour of read/write bandwidth cycles so the
+ * calculation of the performance information can be re-used with different memory homing.
+ */
+class PerformanceInfo : public ObjectRef {
+ public:
+  PerformanceInfo(size_t compute_cycles, std::vector<size_t> read_bytes, size_t write_bytes) {
+    auto n = make_object<PerformanceInfoNode>();
+    n->compute_cycles = compute_cycles;
+    n->read_bytes = std::move(read_bytes);
+    n->write_bytes = write_bytes;
+    data_ = std::move(n);
+  }
+
+  TVM_DEFINE_OBJECT_REF_METHODS(PerformanceInfo, ObjectRef, PerformanceInfoNode);
+};
+
+/*! \brief Node to represent a Tensor */
+class TensorNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \return The shape of the tensor */

Review comment:
       They reflect the members into Python.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] jacobbohlin commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
jacobbohlin commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772966101



##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.
+ * \note The performance information for a Part is composed of 3 factors: the compute cycles,
+ * the number of bytes read from each input tensor and the number of bytes written to the output
+ * tensor. Bytes read/written is reported in favour of read/write bandwidth cycles so the
+ * calculation of the performance information can be re-used with different memory homing.
+ */
+class PerformanceInfo : public ObjectRef {
+ public:
+  PerformanceInfo(size_t compute_cycles, std::vector<size_t> read_bytes, size_t write_bytes) {
+    auto n = make_object<PerformanceInfoNode>();
+    n->compute_cycles = compute_cycles;
+    n->read_bytes = std::move(read_bytes);
+    n->write_bytes = write_bytes;
+    data_ = std::move(n);
+  }
+
+  TVM_DEFINE_OBJECT_REF_METHODS(PerformanceInfo, ObjectRef, PerformanceInfoNode);
+};
+
+/*! \brief Node to represent a Tensor */
+class TensorNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \return The shape of the tensor */

Review comment:
       The reason I'm asking is because PerformanceInfo reflects its members by making them public and wondered why it's done differently here. Is it required for immutability since Tensor has a MUTABLE_OBJECT_REF?

##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.

Review comment:
       Sounds good.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#issuecomment-1005583775


   @leandron @Mousius could you please take a look and merge if you think this is OK to go in? Thanks :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] Mousius merged pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
Mousius merged pull request #9469:
URL: https://github.com/apache/tvm/pull/9469


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#issuecomment-983685540


   cc @JacobBohlin @NicolaLancellotti @csullivan @manupa-arm 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] jacobbohlin commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
jacobbohlin commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772546702



##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.

Review comment:
       ```suggestion
    * \brief A class to hold the performance information for a Part.
   ```

##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.
+ * \note The performance information for a Part is composed of 3 factors: the compute cycles,
+ * the number of bytes read from each input tensor and the number of bytes written to the output
+ * tensor. Bytes read/written is reported in favour of read/write bandwidth cycles so the
+ * calculation of the performance information can be re-used with different memory homing.
+ */
+class PerformanceInfo : public ObjectRef {
+ public:
+  PerformanceInfo(size_t compute_cycles, std::vector<size_t> read_bytes, size_t write_bytes) {
+    auto n = make_object<PerformanceInfoNode>();
+    n->compute_cycles = compute_cycles;
+    n->read_bytes = std::move(read_bytes);
+    n->write_bytes = write_bytes;
+    data_ = std::move(n);
+  }
+
+  TVM_DEFINE_OBJECT_REF_METHODS(PerformanceInfo, ObjectRef, PerformanceInfoNode);
+};
+
+/*! \brief Node to represent a Tensor */
+class TensorNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \return The shape of the tensor */

Review comment:
       What's the purpose of all these functions, as opposed to having public class members? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772974832



##########
File path: cmake/modules/contrib/EthosU.cmake
##########
@@ -18,7 +18,8 @@
 if(USE_ETHOSU)
   file(GLOB COMPILER_ETHOSU_SRCS
        CONFIGURE_DEPENDS src/relay/backend/contrib/ethosu/*
-       CONFIGURE_DEPENDS src/contrib/ethosu/cascader/*)
+       CONFIGURE_DEPENDS src/contrib/ethosu/cascader/*

Review comment:
       If we have source files that defines object definitions, we need to include the irrespective of cmake config.
   See here : https://github.com/apache/tvm/pull/9630

##########
File path: python/tvm/contrib/ethosu/cascader/graph.py
##########
@@ -0,0 +1,170 @@
+# 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.
+"""Graph objects to define compute graphs for the NPU cascader."""
+from typing import List
+from collections import namedtuple
+import tvm._ffi
+
+from tvm.runtime import Object
+
+from .stripe_config import StripeConfig
+from . import _ffi_api
+
+
+TESubgraph = namedtuple("TESubgraph", ["input_tensors", "output_tensor"])
+
+
+@tvm._ffi.register_object("contrib.ethosu.cascader.PerformanceInfo")
+class PerformanceInfo(Object):

Review comment:
       As mentioned in the CMake file, the c-source that defines the object needs to be compiled irrespective of cmake config and this applies to other Object definitions as well.

##########
File path: src/contrib/ethosu/cascader/graph.cc
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.
+ */
+#include "graph.h"
+
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container/array.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <stack>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "common.h"
+#include "stripe_config.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+void PerformanceInfoNode::VisitAttrs(AttrVisitor* v) {
+  v->Visit("_compute_cycles", &compute_cycles);
+  Array<Integer> tmp_reads = make_array(read_bytes);
+  v->Visit("_read_bytes", &tmp_reads);
+  v->Visit("_write_bytes", &write_bytes);
+}
+
+TVM_REGISTER_NODE_TYPE(PerformanceInfoNode);

Review comment:
       It is good if we can attach TVM_STATIC_IR_FUNCTOR(ReprPrinter, vtable) to be able to print these from python side for debugging. We could do that in a follow up.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772707306



##########
File path: src/contrib/ethosu/cascader/common.h
##########
@@ -103,6 +119,18 @@ inline std::size_t hash_vector(const std::vector<T>& vec) {
   return seed;
 }
 
+template <class T>
+inline T mul_reduce(const std::vector<T>& vec) {
+  if (vec.size() == 0) {
+    return 0;
+  }
+  T v = vec[0];
+  for (unsigned int i = 1; i < vec.size(); i++) {
+    v *= vec[i];
+  }
+  return v;
+}

Review comment:
       Good point, could we take in a follow-up/add on to avoid CI respin?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] mbaret commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772707432



##########
File path: src/contrib/ethosu/cascader/graph.h
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/contrib/ethosu/cascader/graph.h
+ * \brief Graph objects (Tensor and Part) for the Ethos-U cascader
+ */
+#ifndef TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+#define TVM_CONTRIB_ETHOSU_CASCADER_GRAPH_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/object.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/tensor.h>
+
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "propagator.h"
+
+namespace tvm {
+namespace contrib {
+namespace ethosu {
+namespace cascader {
+
+class Tensor;
+class Part;
+class StripeConfig;
+
+/*! \brief A struct to hold a Tensor Expression subgraph */
+struct TESubgraph {
+  /*! \brief The input te::Tensors to the subgraph */
+  std::vector<te::Tensor> input_tensors;
+  /*! \brief The output te::Tensor of the subgraph */
+  te::Tensor output_tensor;
+};
+
+/*! \brief Node to hold performance information for a Part */
+class PerformanceInfoNode : public Object {
+ public:
+  void VisitAttrs(AttrVisitor* v);
+
+  /*! \brief The cycles to compute a block */
+  size_t compute_cycles;
+  /*! \brief The number of bytes read per input tensor */
+  std::vector<size_t> read_bytes;
+  /*! \brief The number of bytes written to the output tensor */
+  size_t write_bytes;
+
+  static constexpr const char* _type_key = "contrib.ethosu.cascader.PerformanceInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PerformanceInfoNode, Object);
+};
+
+/*!
+ * \brief An class to hold the performance information for a Part.

Review comment:
       Again, could take this as an addendum to the next PR.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] jacobbohlin commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
jacobbohlin commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772509046



##########
File path: src/contrib/ethosu/cascader/common.h
##########
@@ -103,6 +119,18 @@ inline std::size_t hash_vector(const std::vector<T>& vec) {
   return seed;
 }
 
+template <class T>
+inline T mul_reduce(const std::vector<T>& vec) {
+  if (vec.size() == 0) {
+    return 0;
+  }
+  T v = vec[0];
+  for (unsigned int i = 1; i < vec.size(); i++) {
+    v *= vec[i];
+  }
+  return v;
+}

Review comment:
       Minor suggestion but you could use instead of this template.
   ```C++
   std::accumulate(vec.begin(),vec.end(), 1, std::multiplies<T>());
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] jacobbohlin commented on a change in pull request #9469: [microNPU][2a] Add CascaderGraph for cascading analysis

Posted by GitBox <gi...@apache.org>.
jacobbohlin commented on a change in pull request #9469:
URL: https://github.com/apache/tvm/pull/9469#discussion_r772958219



##########
File path: src/contrib/ethosu/cascader/common.h
##########
@@ -103,6 +119,18 @@ inline std::size_t hash_vector(const std::vector<T>& vec) {
   return seed;
 }
 
+template <class T>
+inline T mul_reduce(const std::vector<T>& vec) {
+  if (vec.size() == 0) {
+    return 0;
+  }
+  T v = vec[0];
+  for (unsigned int i = 1; i < vec.size(); i++) {
+    v *= vec[i];
+  }
+  return v;
+}

Review comment:
       Yes sounds reasonable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org