You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by rahulforallp <gi...@git.apache.org> on 2017/11/08 06:14:50 UTC

[GitHub] carbondata pull request #1473: [WIP]Event listener added

GitHub user rahulforallp opened a pull request:

    https://github.com/apache/carbondata/pull/1473

    [WIP]Event listener added

    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/rahulforallp/incubator-carbondata event_listener

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/1473.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1473
    
----
commit d37e135668f3d11ff8975508f16ada78ef3fb974
Author: rahulforallp <ra...@knoldus.in>
Date:   2017-11-07T06:20:15Z

    Changes for adding event listener interface functionality to Carbondata

commit 3a49399aaa8d5a76aabb51257f7d8739ec78e110
Author: Venkata Ramana G <ra...@huawei.com>
Date:   2017-11-07T12:10:44Z

    Added OperationContext support

commit b5b183f0cae931c79674019c24914d7d2d4cee4a
Author: rahulforallp <ra...@knoldus.in>
Date:   2017-11-07T18:06:26Z

    code refactored

----


---

[GitHub] carbondata issue #1473: [CARBONDATA-1592]Event listener added

Posted by rahulforallp <gi...@git.apache.org>.
Github user rahulforallp commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    @ravipesala jira-ID added and build succeded.


---

[GitHub] carbondata issue #1473: [CARBONDATA-1592]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    LGTM


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/878/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1505/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1482/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149902095
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala ---
    @@ -0,0 +1,143 @@
    +/*
    + * 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 org.apache.carbondata.events
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
    +
    +import org.apache.carbondata.core.metadata.CarbonTableIdentifier
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.events.Event
    +import org.apache.carbondata.processing.loading.model.CarbonLoadModel
    +
    +  /**
    +  * event for database operations
    +  */
    +trait DatabaseEvent extends Event {
    +  val databaseName: String
    +}
    +
    +  /**
    +  * event for table related operations
    +  */
    +trait TableEvent extends DatabaseEvent {
    --- End diff --
    
    Just name as TableEventInfo and don't extend Event


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/874/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/866/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/927/



---

[GitHub] carbondata pull request #1473: [CARBONDATA-1592]Event listener added

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/1473


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/890/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149906040
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala ---
    @@ -46,12 +47,23 @@ case class CleanFilesCommand(
           val relation = catalog
             .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
           val carbonTable = relation.tableMeta.carbonTable
    +      val operationContext = new OperationContext
    +      val cleanFilesPreEvent: CleanFilesPreEvent =
    +        CleanFilesPreEvent(carbonTable,
    +          sparkSession)
    +      OperationListenerBus.getInstance.fireEvent(cleanFilesPreEvent, operationContext)
    --- End diff --
    
    Add one more overload method OperationListenerBus.getInstance.fireEvent(cleanFilesPreEvent) with no operationContext


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    LGTM 
    @rahulforallp Please add the jiraid


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/870/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1541/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149903075
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala ---
    @@ -0,0 +1,143 @@
    +/*
    + * 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 org.apache.carbondata.events
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
    +
    +import org.apache.carbondata.core.metadata.CarbonTableIdentifier
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.events.Event
    +import org.apache.carbondata.processing.loading.model.CarbonLoadModel
    +
    +  /**
    +  * event for database operations
    +  */
    +trait DatabaseEvent extends Event {
    +  val databaseName: String
    +}
    +
    +  /**
    +  * event for table related operations
    +  */
    +trait TableEvent extends DatabaseEvent {
    +  override lazy val databaseName: String = carbonTableIdentifier.getDatabaseName
    +  val carbonTableIdentifier: CarbonTableIdentifier
    +}
    +
    +  /**
    +  * event for load operations
    +  */
    +trait LoadEvent extends TableEvent {
    +  val carbonLoadModel: CarbonLoadModel
    +}
    +
    +  /**
    +  * event for lookup
    +  */
    +trait LookupRelationEvent extends TableEvent {
    +  override val carbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +}
    +
    +
    +  /**
    +  * event for drop table
    +  */
    +trait DropTableEvent extends TableEvent {
    +  override val carbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val ifExistsSet: Boolean
    +}
    +
    +  /**
    +  * event for alter_table_drop_column
    +  */
    +trait AlterTableDropColumnEvent extends TableEvent {
    +  override val carbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val alterTableDropColumnModel: AlterTableDropColumnModel
    +}
    +
    +  /**
    +  * event for alter_table_rename
    +  */
    +trait AlterTableRenameEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val alterTableRenameModel: AlterTableRenameModel
    +}
    +
    +  /**
    +  * event for alter_table_rename
    +  */
    +trait AlterTableCompactionEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val carbonLoadModel: CarbonLoadModel
    +  val mergedLoadName: String
    +}
    +
    +  /**
    +  * event for DeleteSegmentById
    +  */
    +trait DeleteSegmentbyIdEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val loadIds: Seq[String]
    +}
    +
    +  /**
    +  * event for DeleteSegmentByDate
    +  */
    +trait DeleteSegmentbyDateEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +  val loadDates: String
    +}
    +
    +  /**
    +  * event for Clean Files
    +  */
    +trait CleanFilesEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +}
    +
    +  /**
    +  * event for update table
    +  */
    +trait UpdateTableEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +}
    +
    +  /**
    +  * event for delete from table
    +  */
    +trait DeleteFromTableEvent extends TableEvent {
    +  override val carbonTableIdentifier: CarbonTableIdentifier = carbonTable.getCarbonTableIdentifier
    +  val carbonTable: CarbonTable
    +}
    +
    +/**
    + * event to initiate CarbonEnv
    + */
    +trait CarbonEnvEvent extends Event {
    --- End diff --
    
    This also name CarbonEnvEventInfo and don't exend from event


---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149900866
  
    --- Diff: core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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 org.apache.carbondata.events;
    +
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.CopyOnWriteArrayList;
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +
    +/**
    + * An event bus which posts events to its listeners.
    + */
    +public class OperationListenerBus {
    +
    +  /**
    +   * singleton instance
    +   */
    +  private static final OperationListenerBus INSTANCE = new OperationListenerBus();
    +
    +  /**
    +   * Event map to hold all listeners corresponding to an event
    +   */
    +  protected Map<String, List<OperationEventListener>> eventMap =
    +      new ConcurrentHashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
    +
    +  /**
    +   * @return
    +   */
    +  public static OperationListenerBus getInstance() {
    +    return INSTANCE;
    +  }
    +
    +  /**
    +   * Add a listener to listen events. This method is thread-safe and can be called in any thread.
    +   *
    +   * @param eventType
    +   * @param operationEventListener
    +   */
    +  public void addListener(String eventType, OperationEventListener operationEventListener) {
    --- End diff --
    
    Pass the event class instead of String, like Class<? extends Event>


---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149906601
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala ---
    @@ -35,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
     import org.apache.carbondata.core.datamap.DataMapStoreManager
     import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
     
    +
    --- End diff --
    
    Don't change the class without code change


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by rahulforallp <gi...@git.apache.org>.
Github user rahulforallp commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    retest this plese


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1488/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/937/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/893/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/923/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/926/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/873/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1508/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1552/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1542/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/933/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149902025
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala ---
    @@ -0,0 +1,143 @@
    +/*
    + * 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 org.apache.carbondata.events
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
    +
    +import org.apache.carbondata.core.metadata.CarbonTableIdentifier
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.events.Event
    +import org.apache.carbondata.processing.loading.model.CarbonLoadModel
    +
    +  /**
    +  * event for database operations
    +  */
    +trait DatabaseEvent extends Event {
    --- End diff --
    
    Just name as DatabaseEventInfo and don't extend Event


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1548/



---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by rahulforallp <gi...@git.apache.org>.
Github user rahulforallp commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    retest this please


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1492/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149900962
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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 org.apache.carbondata.events
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.SQLContext
    +import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
    +
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.processing.loading.model.CarbonLoadModel
    +
    +/**
    + *
    + */
    +case class AlterTableDropColumnPreEvent(carbonTable: CarbonTable,
    +    alterTableDropColumnModel: AlterTableDropColumnModel,
    +    sparkSession: SparkSession) extends AlterTableDropColumnEvent {
    +  /**
    +   * Method for getting the event type. Used for invoking all listeners registered for an event
    +   *
    +   * @return
    +   */
    +  override def getEventType: String = {
    +    AlterTableDropColumnPreEvent.eventType
    +  }
    +}
    +
    +
    +case class AlterTableDropColumnPostEvent(carbonTable: CarbonTable,
    +    alterTableDropColumnModel: AlterTableDropColumnModel,
    +    sparkSession: SparkSession) extends AlterTableDropColumnEvent {
    +  /**
    +   * Method for getting the event type. Used for invoking all listeners registered for an event
    +   *
    +   * @return
    +   */
    +  override def getEventType: String = {
    --- End diff --
    
    Remoe the eventType and getEventType from all classes


---

[GitHub] carbondata issue #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1473
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1537/



---

[GitHub] carbondata pull request #1473: [WIP]Event listener added

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1473#discussion_r149902256
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala ---
    @@ -0,0 +1,143 @@
    +/*
    + * 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 org.apache.carbondata.events
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
    +
    +import org.apache.carbondata.core.metadata.CarbonTableIdentifier
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.events.Event
    +import org.apache.carbondata.processing.loading.model.CarbonLoadModel
    +
    +  /**
    +  * event for database operations
    +  */
    +trait DatabaseEvent extends Event {
    +  val databaseName: String
    +}
    +
    +  /**
    +  * event for table related operations
    +  */
    +trait TableEvent extends DatabaseEvent {
    +  override lazy val databaseName: String = carbonTableIdentifier.getDatabaseName
    +  val carbonTableIdentifier: CarbonTableIdentifier
    +}
    +
    +  /**
    +  * event for load operations
    +  */
    +trait LoadEvent extends TableEvent {
    --- End diff --
    
    Extend both Event and TableEventInfo


---